Browse Source

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

James 9 năm trước cách đây
mục cha
commit
11671c2bf6

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

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h → hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h

@@ -15,8 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#ifndef LIBHDFSPP_HDFS_H_
-#define LIBHDFSPP_HDFS_H_
+#ifndef LIBHDFSPP_HDFSPP_H_
+#define LIBHDFSPP_HDFSPP_H_
 
 #include "libhdfspp/options.h"
 #include "libhdfspp/status.h"

+ 25 - 0
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;

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

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

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

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

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