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 jh...@apache.org on 2016/01/10 05:53:27 UTC

hadoop git commit: HDFS-9627. libhdfs++: Add mechanism to retrieve human readable error messages through the C API. Contributed by James Clampffer.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-8707 2f207908d -> 1732e7f0a


HDFS-9627. libhdfs++: Add mechanism to retrieve human readable error messages through the C API.  Contributed by James Clampffer.


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

Branch: refs/heads/HDFS-8707
Commit: 1732e7f0adcd273bf00887102a1edd7d8367a3bf
Parents: 2f20790
Author: James <jh...@apache.org>
Authored: Sat Jan 9 23:52:45 2016 -0500
Committer: James <jh...@apache.org>
Committed: Sat Jan 9 23:52:45 2016 -0500

----------------------------------------------------------------------
 .../native/libhdfspp/include/libhdfspp/hdfs.h   | 148 -------------------
 .../libhdfspp/include/libhdfspp/hdfs_ext.h      |  67 +++++++++
 .../native/libhdfspp/include/libhdfspp/hdfspp.h | 148 +++++++++++++++++++
 .../native/libhdfspp/lib/bindings/c/hdfs.cc     |  25 ++++
 .../libhdfspp/lib/common/hdfs_public_api.h      |   2 +-
 .../libhdfspp/lib/fs/bad_datanode_tracker.h     |   2 +-
 .../main/native/libhdfspp/lib/fs/filesystem.h   |   2 +-
 .../main/native/libhdfspp/tests/CMakeLists.txt  |   6 +-
 .../native/libhdfspp/tests/hdfspp_errors.cc     | 117 +++++++++++++++
 9 files changed, 365 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h
deleted file mode 100644
index 7dc3f88..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef LIBHDFSPP_HDFS_H_
-#define LIBHDFSPP_HDFS_H_
-
-#include "libhdfspp/options.h"
-#include "libhdfspp/status.h"
-
-#include <functional>
-#include <memory>
-#include <set>
-#include <iostream>
-
-namespace hdfs {
-
-/**
- * An IoService manages a queue of asynchronous tasks. All libhdfs++
- * operations are filed against a particular IoService.
- *
- * When an operation is queued into an IoService, the IoService will
- * run the callback handler associated with the operation. Note that
- * the IoService must be stopped before destructing the objects that
- * file the operations.
- *
- * From an implementation point of view the IoService object wraps the
- * ::asio::io_service objects. Please see the related documentation
- * for more details.
- **/
-class IoService {
- public:
-  static IoService *New();
-  /**
-   * Run the asynchronous tasks associated with this IoService.
-   **/
-  virtual void Run() = 0;
-  /**
-   * Stop running asynchronous tasks associated with this IoService.
-   **/
-  virtual void Stop() = 0;
-  virtual ~IoService();
-};
-
-/**
- * A node exclusion rule provides a simple way of testing if the
- * client should attempt to connect to a node based on the node's
- * UUID.  The FileSystem and FileHandle use the BadDataNodeTracker
- * by default.  AsyncPreadSome takes an optional NodeExclusionRule
- * that will override the BadDataNodeTracker.
- **/
-class NodeExclusionRule {
- public:
-  virtual ~NodeExclusionRule(){};
-  virtual bool IsBadNode(const std::string &node_uuid) = 0;
-};
-
-/**
- * Applications opens a FileHandle to read files in HDFS.
- **/
-class FileHandle {
-public:
-  /**
-   * Read data from a specific position. The current implementation
-   * stops at the block boundary.
-   *
-   * @param buf the pointer to the buffer
-   * @param nbyte the size of the buffer
-   * @param offset the offset the file
-   *
-   * The handler returns the datanode that serves the block and the number of
-   * bytes has read.
-   **/
-  virtual void
-  PositionRead(void *buf, size_t nbyte, uint64_t offset,
-               const std::function<void(const Status &, size_t)> &handler) = 0;
-
-  virtual Status PositionRead(void *buf, size_t *nbyte, off_t offset) = 0;
-  virtual Status Read(void *buf, size_t *nbyte) = 0;
-  virtual Status Seek(off_t *offset, std::ios_base::seekdir whence) = 0;
-
-  /**
-   * Determine if a datanode should be excluded from future operations
-   * based on the return Status.
-   *
-   * @param status the Status object returned by InputStream::PositionRead
-   * @return true if the status indicates a failure that is not recoverable
-   * by the client and false otherwise.
-   **/
-  static bool ShouldExclude(const Status &status);
-
-  virtual ~FileHandle();
-};
-
-/**
- * FileSystem implements APIs to interact with HDFS.
- **/
-class FileSystem {
- public:
-  /**
-   * Create a new instance of the FileSystem object. The call
-   * initializes the RPC connections to the NameNode and returns an
-   * FileSystem object.
-   **/
-  static FileSystem * New(
-      IoService *&io_service, const Options &options);
-
-  virtual void Connect(const std::string &server,
-      const std::string &service,
-      const std::function<void(const Status &, FileSystem *)> &&handler) = 0;
-
-  /* Synchronous call of Connect */
-  virtual Status Connect(const std::string &server,
-      const std::string &service) = 0;
-
-  /**
-   * Open a file on HDFS. The call issues an RPC to the NameNode to
-   * gather the locations of all blocks in the file and to return a
-   * new instance of the @ref InputStream object.
-   **/
-  virtual void
-  Open(const std::string &path,
-       const std::function<void(const Status &, FileHandle *)> &handler) = 0;
-  virtual Status Open(const std::string &path, FileHandle **handle) = 0;
-
-  /**
-   * Note that it is an error to destroy the filesystem from within a filesystem
-   * callback.  It will lead to a deadlock and the termination of the process.
-   */
-  virtual ~FileSystem() {};
-
-};
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h
new file mode 100644
index 0000000..2d793f4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h
@@ -0,0 +1,67 @@
+/**
+ * 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 LIBHDFSPP_HDFS_HDFSEXT
+#define LIBHDFSPP_HDFS_HDFSEXT
+
+/* get typdefs and #defines from libhdfs' hdfs.h to stay consistent */
+#include <hdfs/hdfs.h>
+
+/**
+ *  Note: The #defines below are copied directly from libhdfs'
+ *  hdfs.h.  LIBHDFS_EXTERNAL gets explicitly #undefed at the
+ *  end of the file so it must be redefined here.
+ **/
+
+#ifdef WIN32
+    #ifdef LIBHDFS_DLL_EXPORT
+        #define LIBHDFS_EXTERNAL __declspec(dllexport)
+    #elif LIBHDFS_DLL_IMPORT
+        #define LIBHDFS_EXTERNAL __declspec(dllimport)
+    #else
+        #define LIBHDFS_EXTERNAL
+    #endif
+#else
+    #ifdef LIBHDFS_DLL_EXPORT
+        #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
+    #elif LIBHDFS_DLL_IMPORT
+        #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
+    #else
+        #define LIBHDFS_EXTERNAL
+    #endif
+#endif
+
+
+/**
+ * Keep C bindings that are libhdfs++ specific in here.
+ **/
+
+extern "C" {
+/**
+ *  Reads the last error, if any, that happened in this thread
+ *  into the user supplied buffer.
+ *  @param buf  A chunk of memory with room for the error string.
+ *  @param len  Size of the buffer, if the message is longer than
+ *              len len-1 bytes of the message will be copied.
+ **/
+
+LIBHDFS_EXTERNAL
+void hdfsGetLastError(char *buf, int len);
+
+
+} /* end extern "C" */
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h
new file mode 100644
index 0000000..ebcf227
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h
@@ -0,0 +1,148 @@
+/**
+ * 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 LIBHDFSPP_HDFSPP_H_
+#define LIBHDFSPP_HDFSPP_H_
+
+#include "libhdfspp/options.h"
+#include "libhdfspp/status.h"
+
+#include <functional>
+#include <memory>
+#include <set>
+#include <iostream>
+
+namespace hdfs {
+
+/**
+ * An IoService manages a queue of asynchronous tasks. All libhdfs++
+ * operations are filed against a particular IoService.
+ *
+ * When an operation is queued into an IoService, the IoService will
+ * run the callback handler associated with the operation. Note that
+ * the IoService must be stopped before destructing the objects that
+ * file the operations.
+ *
+ * From an implementation point of view the IoService object wraps the
+ * ::asio::io_service objects. Please see the related documentation
+ * for more details.
+ **/
+class IoService {
+ public:
+  static IoService *New();
+  /**
+   * Run the asynchronous tasks associated with this IoService.
+   **/
+  virtual void Run() = 0;
+  /**
+   * Stop running asynchronous tasks associated with this IoService.
+   **/
+  virtual void Stop() = 0;
+  virtual ~IoService();
+};
+
+/**
+ * A node exclusion rule provides a simple way of testing if the
+ * client should attempt to connect to a node based on the node's
+ * UUID.  The FileSystem and FileHandle use the BadDataNodeTracker
+ * by default.  AsyncPreadSome takes an optional NodeExclusionRule
+ * that will override the BadDataNodeTracker.
+ **/
+class NodeExclusionRule {
+ public:
+  virtual ~NodeExclusionRule(){};
+  virtual bool IsBadNode(const std::string &node_uuid) = 0;
+};
+
+/**
+ * Applications opens a FileHandle to read files in HDFS.
+ **/
+class FileHandle {
+public:
+  /**
+   * Read data from a specific position. The current implementation
+   * stops at the block boundary.
+   *
+   * @param buf the pointer to the buffer
+   * @param nbyte the size of the buffer
+   * @param offset the offset the file
+   *
+   * The handler returns the datanode that serves the block and the number of
+   * bytes has read.
+   **/
+  virtual void
+  PositionRead(void *buf, size_t nbyte, uint64_t offset,
+               const std::function<void(const Status &, size_t)> &handler) = 0;
+
+  virtual Status PositionRead(void *buf, size_t *nbyte, off_t offset) = 0;
+  virtual Status Read(void *buf, size_t *nbyte) = 0;
+  virtual Status Seek(off_t *offset, std::ios_base::seekdir whence) = 0;
+
+  /**
+   * Determine if a datanode should be excluded from future operations
+   * based on the return Status.
+   *
+   * @param status the Status object returned by InputStream::PositionRead
+   * @return true if the status indicates a failure that is not recoverable
+   * by the client and false otherwise.
+   **/
+  static bool ShouldExclude(const Status &status);
+
+  virtual ~FileHandle();
+};
+
+/**
+ * FileSystem implements APIs to interact with HDFS.
+ **/
+class FileSystem {
+ public:
+  /**
+   * Create a new instance of the FileSystem object. The call
+   * initializes the RPC connections to the NameNode and returns an
+   * FileSystem object.
+   **/
+  static FileSystem * New(
+      IoService *&io_service, const Options &options);
+
+  virtual void Connect(const std::string &server,
+      const std::string &service,
+      const std::function<void(const Status &, FileSystem *)> &&handler) = 0;
+
+  /* Synchronous call of Connect */
+  virtual Status Connect(const std::string &server,
+      const std::string &service) = 0;
+
+  /**
+   * Open a file on HDFS. The call issues an RPC to the NameNode to
+   * gather the locations of all blocks in the file and to return a
+   * new instance of the @ref InputStream object.
+   **/
+  virtual void
+  Open(const std::string &path,
+       const std::function<void(const Status &, FileHandle *)> &handler) = 0;
+  virtual Status Open(const std::string &path, FileHandle **handle) = 0;
+
+  /**
+   * Note that it is an error to destroy the filesystem from within a filesystem
+   * callback.  It will lead to a deadlock and the termination of the process.
+   */
+  virtual ~FileSystem() {};
+
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
index d23c7b0..0fc02b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
@@ -19,9 +19,12 @@
 #include "fs/filesystem.h"
 
 #include <hdfs/hdfs.h>
+#include <libhdfspp/hdfs_ext.h>
+
 #include <string>
 #include <cstring>
 #include <iostream>
+#include <algorithm>
 
 using namespace hdfs;
 
@@ -49,9 +52,31 @@ struct hdfsFile_internal {
   std::unique_ptr<FileHandle> file_;
 };
 
+/* Keep thread local copy of last error string */
+thread_local std::string errstr;
+
+/* Fetch last error that happened in this thread */
+void hdfsGetLastError(char *buf, int len) {
+  if(nullptr == buf || len < 1 || errstr.empty()) {
+    return;
+  }
+
+  /* leave space for a trailing null */
+  size_t copylen = std::min((size_t)errstr.size(), (size_t)len);
+  if(copylen == (size_t)len) {
+    copylen--;
+  }
+
+  strncpy(buf, errstr.c_str(), copylen);
+
+  /* stick in null */
+  buf[copylen] = 0;
+}
+
 /* Error handling with optional debug to stderr */
 static void ReportError(int errnum, std::string msg) {
   errno = errnum;
+  errstr = msg;
 #ifdef LIBHDFSPP_C_API_ENABLE_DEBUG
   std::cerr << "Error: errno=" << strerror(errnum) << " message=\"" << msg
             << "\"" << std::endl;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
index 95567c0..acd96024 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
@@ -19,7 +19,7 @@
 #ifndef COMMON_HDFS_PUBLIC_API_H_
 #define COMMON_HDFS_PUBLIC_API_H_
 
-#include "libhdfspp/hdfs.h"
+#include "libhdfspp/hdfspp.h"
 
 #include <asio/io_service.hpp>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
index f565192..72855dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
@@ -26,7 +26,7 @@
 #include <set>
 
 #include "libhdfspp/options.h"
-#include "libhdfspp/hdfs.h"
+#include "libhdfspp/hdfspp.h"
 
 namespace hdfs {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
index d78df81..df61872 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
@@ -21,7 +21,7 @@
 #include "filehandle.h"
 #include "common/hdfs_public_api.h"
 #include "common/async_stream.h"
-#include "libhdfspp/hdfs.h"
+#include "libhdfspp/hdfspp.h"
 #include "fs/bad_datanode_tracker.h"
 #include "rpc/rpc_engine.h"
 #include "reader/block_reader.h"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
index 3dee801..b1d0989 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
@@ -86,7 +86,11 @@ add_memcheck_test(configuration configuration_test)
 
 add_executable(hdfs_configuration_test hdfs_configuration_test.cc)
 target_link_libraries(hdfs_configuration_test common gmock_main ${CMAKE_THREAD_LIBS_INIT})
-add_test(hdfs_configuration hdfs_configuration_test)
+add_memcheck_test(hdfs_configuration hdfs_configuration_test)
+
+add_executable(hdfspp_errors_test hdfspp_errors.cc)
+target_link_libraries(hdfspp_errors_test common gmock_main bindings_c fs rpc proto common reader connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT})
+add_memcheck_test(hdfspp_errors hdfspp_errors_test)
 
 #This test requires a great deal of Hadoop Java infrastructure to run.
 if(HADOOP_BUILD)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1732e7f0/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc
new file mode 100644
index 0000000..0cb6545
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.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 <hdfs/hdfs.h>
+#include <libhdfspp/hdfs_ext.h>
+
+#include <google/protobuf/io/coded_stream.h>
+#include <gmock/gmock.h>
+
+#include <string.h>
+#include <string>
+
+using ::testing::_;
+using ::testing::InvokeArgument;
+using ::testing::Return;
+
+/* Don't need a real minidfs cluster since this just passes invalid params. */
+
+TEST(HdfsppErrors, NullFileSystem) {
+
+  char buf[4096];
+
+  hdfsFS fs = nullptr;
+  hdfsFile fd = reinterpret_cast<hdfsFile>(1);
+
+  tSize res = hdfsRead(fs, fd, buf, 4096);
+  ASSERT_EQ(res, -1);
+
+  hdfsGetLastError(buf, 4096);
+
+  ASSERT_EQ(std::string(buf), "Cannot perform FS operations with null FS handle.");
+}
+
+TEST(HdfsppErrors, NullFileHandle) {
+  char buf[4096];
+
+  hdfsFS fs = reinterpret_cast<hdfsFS>(1);
+  hdfsFile fd = nullptr;
+
+  tSize res = hdfsRead(fs, fd, buf, 4096);
+  ASSERT_EQ(res, -1);
+
+  hdfsGetLastError(buf, 4096);
+
+  ASSERT_EQ(std::string(buf), "Cannot perform FS operations with null File handle.");
+}
+
+TEST(HdfsppErrors, ZeroLength) {
+  char buf[1];
+  buf[0] = 0;
+
+  hdfsFS fs = reinterpret_cast<hdfsFS>(1);
+  hdfsFile fd = nullptr;
+
+  tSize res = hdfsRead(fs, fd, buf, 1);
+  ASSERT_EQ(res, -1);
+
+  hdfsGetLastError(buf, 0);
+
+  ASSERT_EQ(std::string(buf), "");
+}
+
+TEST(HdfsppErrors, NegativeLength) {
+  char buf[1];
+  buf[0] = 0;
+
+  hdfsFS fs = reinterpret_cast<hdfsFS>(1);
+  hdfsFile fd = nullptr;
+
+  tSize res = hdfsRead(fs, fd, buf, 1);
+  ASSERT_EQ(res, -1);
+
+  hdfsGetLastError(buf, -1);
+
+  ASSERT_EQ(std::string(buf), "");
+}
+
+TEST(HdfsppErrors, MessageTruncation) {
+  char buf[4096];
+
+  hdfsFS fs = reinterpret_cast<hdfsFS>(1);
+  hdfsFile fd = nullptr;
+
+  tSize res = hdfsRead(fs, fd, buf, 4096);
+  ASSERT_EQ(res, -1);
+
+  hdfsGetLastError(buf, 10);
+
+  ASSERT_EQ(std::string(buf), "Cannot pe");
+}
+
+int main(int argc, char *argv[]) {
+  // The following line must be executed to initialize Google Mock
+  // (and Google Test) before running the tests.
+  ::testing::InitGoogleMock(&argc, argv);
+  int exit_code = RUN_ALL_TESTS();
+  google::protobuf::ShutdownProtobufLibrary();
+
+  return exit_code;
+}
+