Quellcode durchsuchen

HDFS-9359. Test libhdfs++ with existing libhdfs tests. Contributed by Stephen Walkauskas.

James vor 9 Jahren
Ursprung
Commit
a38703fdfc

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt

@@ -16,6 +16,24 @@
 # limitations under the License.
 #
 
+# Delegate some functionality to libhdfs, until libhdfspp is complete.
+set (LIBHDFS_SRC_DIR ../../libhdfs)
+set (LIBHDFS_TESTS_DIR ../../libhdfs-tests)
+set (LIBHDFSPP_SRC_DIR ..)
+set (LIBHDFSPP_LIB_DIR ${LIBHDFSPP_SRC_DIR}/lib)
+set (LIBHDFSPP_BINDING_C ${LIBHDFSPP_LIB_DIR}/bindings/c)
+include_directories(
+    ${GENERATED_JAVAH}
+    ${CMAKE_CURRENT_SOURCE_DIR}
+    ${CMAKE_BINARY_DIR}
+    ${JNI_INCLUDE_DIRS}
+    ${LIBHDFS_SRC_DIR}/include
+    ${LIBHDFS_SRC_DIR}/..
+    ${LIBHDFS_SRC_DIR}
+    ${OS_DIR}
+)
+add_library(hdfspp_test_shim_static STATIC hdfs_shim.c libhdfs_wrapper.c libhdfspp_wrapper.cc ${LIBHDFSPP_BINDING_C}/hdfs_cpp.cc)
+
 add_library(test_common OBJECT mock_connection.cc)
 
 set(PROTOBUF_IMPORT_DIRS ${PROTO_HADOOP_TEST_DIR})
@@ -49,3 +67,7 @@ add_test(bad_datanode bad_datanode_test)
 add_executable(node_exclusion_test node_exclusion_test.cc)
 target_link_libraries(node_exclusion_test fs gmock_main common ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} ${CMAKE_THREAD_LIBS_INIT})
 add_test(node_exclusion node_exclusion_test)
+
+build_libhdfs_test(libhdfs_threaded hdfspp_test_shim_static expect.c test_libhdfs_threaded.c ${OS_DIR}/thread.c)
+link_libhdfs_test(libhdfs_threaded hdfspp_test_shim_static fs reader rpc proto common ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY})
+add_libhdfs_test(libhdfs_threaded hdfspp_test_shim_static)

+ 381 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c

@@ -0,0 +1,381 @@
+/**
+ * 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 "libhdfs_wrapper.h"
+#include "libhdfspp_wrapper.h"
+#include "hdfs/hdfs.h"
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+/* Cheat for now and use the same hdfsBuilder as libhdfs */
+/* (libhdfspp doesn't have an hdfsBuilder yet). */
+struct hdfsBuilder {
+    int forceNewInstance;
+    const char *nn;
+    tPort port;
+    const char *kerbTicketCachePath;
+    const char *userName;
+    struct hdfsBuilderConfOpt *opts;
+};
+
+/* Shim structs and functions that delegate to libhdfspp and libhdfs. */
+struct hdfs_internal {
+  libhdfs_hdfsFS libhdfsRep;
+  libhdfspp_hdfsFS libhdfsppRep;
+};
+typedef struct hdfs_internal* hdfsFS;
+
+struct hdfsFile_internal {
+  libhdfs_hdfsFile libhdfsRep;
+  libhdfspp_hdfsFile libhdfsppRep;
+};
+typedef struct hdfsFile_internal* hdfsFile;
+
+#define REPORT_FUNCTION_NOT_IMPLEMENTED                     \
+  fprintf(stderr, "%s failed: function not implemented by " \
+    "libhdfs++ test shim", __PRETTY_FUNCTION__);
+
+int hdfsFileIsOpenForWrite(hdfsFile file) {
+  return libhdfs_hdfsFileIsOpenForWrite(file->libhdfsRep);
+}
+
+int hdfsFileGetReadStatistics(hdfsFile file,
+                              struct hdfsReadStatistics **stats) {
+  return libhdfs_hdfsFileGetReadStatistics
+      (file->libhdfsRep, (struct libhdfs_hdfsReadStatistics **)stats);
+}
+
+int64_t hdfsReadStatisticsGetRemoteBytesRead(
+                        const struct hdfsReadStatistics *stats) {
+  return libhdfs_hdfsReadStatisticsGetRemoteBytesRead
+      ((struct libhdfs_hdfsReadStatistics *)stats);
+}
+
+int hdfsFileClearReadStatistics(hdfsFile file) {
+  return libhdfs_hdfsFileClearReadStatistics(file->libhdfsRep);
+}
+
+void hdfsFileFreeReadStatistics(struct hdfsReadStatistics *stats) {
+  libhdfs_hdfsFileFreeReadStatistics(
+      (struct libhdfs_hdfsReadStatistics *)stats);
+}
+
+hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user) {
+  REPORT_FUNCTION_NOT_IMPLEMENTED
+  return NULL;
+}
+
+hdfsFS hdfsConnect(const char* nn, tPort port) {
+  REPORT_FUNCTION_NOT_IMPLEMENTED
+  return NULL;
+}
+
+hdfsFS hdfsConnectAsUserNewInstance(const char* nn, tPort port, const char *user ) {
+  REPORT_FUNCTION_NOT_IMPLEMENTED
+  return NULL;
+}
+
+hdfsFS hdfsConnectNewInstance(const char* nn, tPort port) {
+  REPORT_FUNCTION_NOT_IMPLEMENTED
+  return NULL;
+}
+
+hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld) {
+  hdfsFS ret = calloc(1, sizeof(struct hdfs_internal));
+  ret->libhdfsppRep = libhdfspp_hdfsConnect(bld->nn, bld->port);
+  if (!ret->libhdfsppRep) {
+    free(ret);
+    ret = NULL;
+  } else {
+    /* Destroys bld object. */
+    ret->libhdfsRep = libhdfs_hdfsBuilderConnect(bld);
+    if (!ret->libhdfsRep) {
+      libhdfspp_hdfsDisconnect(ret->libhdfsppRep);
+      free(ret);
+      ret = NULL;
+	}
+  }
+  return ret;
+}
+
+struct hdfsBuilder *hdfsNewBuilder(void) {
+  return libhdfs_hdfsNewBuilder();
+}
+
+void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld) {
+  libhdfs_hdfsBuilderSetForceNewInstance(bld);
+}
+
+void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn) {
+  libhdfs_hdfsBuilderSetNameNode(bld, nn);
+}
+
+void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port) {
+  libhdfs_hdfsBuilderSetNameNodePort(bld, port);
+}
+
+void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName) {
+  libhdfs_hdfsBuilderSetUserName(bld, userName);
+}
+
+void hdfsBuilderSetKerbTicketCachePath(struct hdfsBuilder *bld,
+                               const char *kerbTicketCachePath) {
+  libhdfs_hdfsBuilderSetKerbTicketCachePath(bld, kerbTicketCachePath);
+}
+
+void hdfsFreeBuilder(struct hdfsBuilder *bld) {
+  libhdfs_hdfsFreeBuilder(bld);
+}
+
+int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
+                          const char *val) {
+  return libhdfs_hdfsBuilderConfSetStr(bld, key, val);
+}
+
+int hdfsConfGetStr(const char *key, char **val) {
+  return libhdfs_hdfsConfGetStr(key, val);
+}
+
+int hdfsConfGetInt(const char *key, int32_t *val) {
+  return libhdfs_hdfsConfGetInt(key, val);
+}
+
+void hdfsConfStrFree(char *val) {
+  libhdfs_hdfsConfStrFree(val);
+}
+
+int hdfsDisconnect(hdfsFS fs) {
+  int ret;
+  libhdfspp_hdfsDisconnect(fs->libhdfsppRep);
+  ret = libhdfs_hdfsDisconnect(fs->libhdfsRep);
+  free(fs);
+  return ret;
+}
+
+hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
+                      int bufferSize, short replication, tSize blocksize) {
+  hdfsFile ret = calloc(1, sizeof(struct hdfsFile_internal));
+  /* Currently only open libhdf++ for reads. */
+  ret->libhdfsppRep = 0;
+  if (flags == O_RDONLY) {
+    ret->libhdfsppRep = libhdfspp_hdfsOpenFile(fs->libhdfsppRep, path, flags,
+        bufferSize, replication, blocksize);
+  }
+  ret->libhdfsRep = libhdfs_hdfsOpenFile(fs->libhdfsRep, path,
+      flags, bufferSize, replication, blocksize);
+  if (!ret->libhdfsRep) {
+    free(ret);
+    ret = NULL;
+  }
+  return ret;
+}
+
+int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength) {
+  return libhdfs_hdfsTruncateFile(fs->libhdfsRep, path, newlength);
+}
+
+int hdfsUnbufferFile(hdfsFile file) {
+  return libhdfs_hdfsUnbufferFile(file->libhdfsRep);
+}
+
+int hdfsCloseFile(hdfsFS fs, hdfsFile file) {
+  int ret;
+  if (file->libhdfsppRep) {
+    libhdfspp_hdfsCloseFile(fs->libhdfsppRep, file->libhdfsppRep);
+  }
+  ret = libhdfs_hdfsCloseFile(fs->libhdfsRep, file->libhdfsRep);
+  free(file);
+  return ret;
+}
+
+int hdfsExists(hdfsFS fs, const char *path) {
+  return libhdfs_hdfsExists(fs->libhdfsRep, path);
+}
+
+int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos) {
+  return libhdfs_hdfsSeek(fs->libhdfsRep, file->libhdfsRep, desiredPos);
+}
+
+tOffset hdfsTell(hdfsFS fs, hdfsFile file) {
+  return libhdfs_hdfsTell(fs->libhdfsRep, file->libhdfsRep);
+}
+
+tSize hdfsRead(hdfsFS fs, hdfsFile file, void* buffer, tSize length) {
+  return libhdfs_hdfsRead(fs->libhdfsRep, file->libhdfsRep, buffer, length);
+}
+
+tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position,
+                void* buffer, tSize length) {
+  tSize ret = -1;
+  if (!fs->libhdfsppRep) {
+    fprintf(stderr, "hdfsPread failed: no libhdfs++ file system");
+  } else if (!file->libhdfsppRep) {
+    fprintf(stderr, "hdfsPread failed: no libhdfs++ file");
+  } else {
+    ret = libhdfspp_hdfsPread(fs->libhdfsppRep, file->libhdfsppRep,
+        position, buffer, length);
+  }
+  return ret;
+}
+
+tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer,
+                tSize length) {
+  return libhdfs_hdfsWrite(fs->libhdfsRep, file->libhdfsRep, buffer, length);
+}
+
+int hdfsFlush(hdfsFS fs, hdfsFile file) {
+  return libhdfs_hdfsFlush(fs->libhdfsRep, file->libhdfsRep);
+}
+
+int hdfsHFlush(hdfsFS fs, hdfsFile file) {
+  return libhdfs_hdfsHFlush(fs->libhdfsRep, file->libhdfsRep);
+}
+
+int hdfsHSync(hdfsFS fs, hdfsFile file) {
+  return libhdfs_hdfsHSync(fs->libhdfsRep, file->libhdfsRep);
+}
+
+int hdfsAvailable(hdfsFS fs, hdfsFile file) {
+  return libhdfs_hdfsAvailable(fs->libhdfsRep, file->libhdfsRep);
+}
+
+int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst) {
+  return libhdfs_hdfsCopy(srcFS->libhdfsRep, src, dstFS->libhdfsRep, dst);
+}
+
+int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst) {
+  return libhdfs_hdfsMove(srcFS->libhdfsRep, src, dstFS->libhdfsRep, dst);
+}
+
+int hdfsDelete(hdfsFS fs, const char* path, int recursive) {
+  return libhdfs_hdfsDelete(fs->libhdfsRep, path, recursive);
+}
+
+int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath) {
+  return libhdfs_hdfsRename(fs->libhdfsRep, oldPath, newPath);
+}
+
+char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize) {
+  return libhdfs_hdfsGetWorkingDirectory(fs->libhdfsRep, buffer, bufferSize);
+}
+
+int hdfsSetWorkingDirectory(hdfsFS fs, const char* path) {
+  return libhdfs_hdfsSetWorkingDirectory(fs->libhdfsRep, path);
+}
+
+int hdfsCreateDirectory(hdfsFS fs, const char* path) {
+  return libhdfs_hdfsCreateDirectory(fs->libhdfsRep, path);
+}
+
+int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication) {
+  return libhdfs_hdfsSetReplication(fs->libhdfsRep, path, replication);
+}
+
+hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path,
+                                int *numEntries) {
+  return (hdfsFileInfo *)libhdfs_hdfsListDirectory(fs->libhdfsRep, path, numEntries);
+}
+
+hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path) {
+  return (hdfsFileInfo *)libhdfs_hdfsGetPathInfo(fs->libhdfsRep, path);
+}
+
+void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries) {
+  return libhdfs_hdfsFreeFileInfo
+      ((libhdfs_hdfsFileInfo *) hdfsFileInfo, numEntries);
+}
+
+int hdfsFileIsEncrypted(hdfsFileInfo *hdfsFileInfo) {
+  return libhdfs_hdfsFileIsEncrypted
+      ((libhdfs_hdfsFileInfo *) hdfsFileInfo);
+}
+
+char*** hdfsGetHosts(hdfsFS fs, const char* path, 
+        tOffset start, tOffset length) {
+  return libhdfs_hdfsGetHosts(fs->libhdfsRep, path, start, length);
+}
+
+void hdfsFreeHosts(char ***blockHosts) {
+  return libhdfs_hdfsFreeHosts(blockHosts);
+}
+
+tOffset hdfsGetDefaultBlockSize(hdfsFS fs) {
+  return libhdfs_hdfsGetDefaultBlockSize(fs->libhdfsRep);
+}
+
+tOffset hdfsGetDefaultBlockSizeAtPath(hdfsFS fs, const char *path) {
+  return libhdfs_hdfsGetDefaultBlockSizeAtPath(fs->libhdfsRep, path);
+}
+
+tOffset hdfsGetCapacity(hdfsFS fs) {
+  return libhdfs_hdfsGetCapacity(fs->libhdfsRep);
+}
+
+tOffset hdfsGetUsed(hdfsFS fs) {
+  return libhdfs_hdfsGetUsed(fs->libhdfsRep);
+}
+
+int hdfsChown(hdfsFS fs, const char* path, const char *owner,
+              const char *group) {
+  return libhdfs_hdfsChown(fs->libhdfsRep, path, owner, group);
+}
+
+int hdfsChmod(hdfsFS fs, const char* path, short mode) {
+  return libhdfs_hdfsChmod(fs->libhdfsRep, path, mode);
+}
+
+int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime) {
+  return libhdfs_hdfsUtime(fs->libhdfsRep, path, mtime, atime);
+}
+
+struct hadoopRzOptions *hadoopRzOptionsAlloc(void) {
+  return libhdfs_hadoopRzOptionsAlloc();
+}
+
+int hadoopRzOptionsSetSkipChecksum(
+        struct hadoopRzOptions *opts, int skip) {
+  return libhdfs_hadoopRzOptionsSetSkipChecksum(opts, skip);
+}
+
+int hadoopRzOptionsSetByteBufferPool(
+        struct hadoopRzOptions *opts, const char *className) {
+  return libhdfs_hadoopRzOptionsSetByteBufferPool(opts, className);
+}
+
+void hadoopRzOptionsFree(struct hadoopRzOptions *opts) {
+  libhdfs_hadoopRzOptionsFree(opts);
+}
+
+struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
+        struct hadoopRzOptions *opts, int32_t maxLength) {
+  return libhdfs_hadoopReadZero(file->libhdfsRep, opts, maxLength);
+}
+
+int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer) {
+  return libhdfs_hadoopRzBufferLength(buffer);
+}
+
+const void *hadoopRzBufferGet(const struct hadoopRzBuffer *buffer) {
+  return libhdfs_hadoopRzBufferGet(buffer);
+}
+
+void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer) {
+  return libhdfs_hadoopRzBufferFree(file->libhdfsRep, buffer);
+}

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper.c

@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+/* Rename libhdfs structs and functions */
+#include "libhdfs_wrapper_defines.h"
+#include "libhdfs/hdfs.c"
+#include "libhdfs_wrapper_undefs.h"

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper.h

@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+/* "Original" symbols can be included elsewhere. */
+#undef LIBHDFS_HDFS_H
+
+/* Rename libhdfs structs and functions */
+#include "libhdfs_wrapper_defines.h"
+#include "hdfs/hdfs.h"
+#include "libhdfs_wrapper_undefs.h"
+
+/* "Original" symbols can be included elsewhere. */
+#undef LIBHDFS_HDFS_H

+ 92 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_defines.h

@@ -0,0 +1,92 @@
+/**
+ * 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.
+ */
+#define hdfsFileIsOpenForRead libhdfs_hdfsFileIsOpenForRead
+#define hdfsFileIsOpenForWrite libhdfs_hdfsFileIsOpenForWrite
+#define hdfsFileGetReadStatistics libhdfs_hdfsFileGetReadStatistics
+#define hdfsReadStatisticsGetRemoteBytesRead libhdfs_hdfsReadStatisticsGetRemoteBytesRead
+#define hdfsFileClearReadStatistics libhdfs_hdfsFileClearReadStatistics
+#define hdfsFileFreeReadStatistics libhdfs_hdfsFileFreeReadStatistics
+#define hdfsConnectAsUser libhdfs_hdfsConnectAsUser
+#define hdfsConnect libhdfs_hdfsConnect
+#define hdfsConnectAsUserNewInstance libhdfs_hdfsConnectAsUserNewInstance
+#define hdfsConnectNewInstance libhdfs_hdfsConnectNewInstance
+#define hdfsBuilderConnect libhdfs_hdfsBuilderConnect
+#define hdfsNewBuilder libhdfs_hdfsNewBuilder
+#define hdfsBuilderSetForceNewInstance libhdfs_hdfsBuilderSetForceNewInstance
+#define hdfsBuilderSetNameNode libhdfs_hdfsBuilderSetNameNode
+#define hdfsBuilderSetNameNodePort libhdfs_hdfsBuilderSetNameNodePort
+#define hdfsBuilderSetUserName libhdfs_hdfsBuilderSetUserName
+#define hdfsBuilderSetKerbTicketCachePath libhdfs_hdfsBuilderSetKerbTicketCachePath
+#define hdfsFreeBuilder libhdfs_hdfsFreeBuilder
+#define hdfsBuilderConfSetStr libhdfs_hdfsBuilderConfSetStr
+#define hdfsConfGetStr libhdfs_hdfsConfGetStr
+#define hdfsConfGetInt libhdfs_hdfsConfGetInt
+#define hdfsConfStrFree libhdfs_hdfsConfStrFree
+#define hdfsDisconnect libhdfs_hdfsDisconnect
+#define hdfsOpenFile libhdfs_hdfsOpenFile
+#define hdfsTruncateFile libhdfs_hdfsTruncateFile
+#define hdfsUnbufferFile libhdfs_hdfsUnbufferFile
+#define hdfsCloseFile libhdfs_hdfsCloseFile
+#define hdfsExists libhdfs_hdfsExists
+#define hdfsSeek libhdfs_hdfsSeek
+#define hdfsTell libhdfs_hdfsTell
+#define hdfsRead libhdfs_hdfsRead
+#define hdfsPread libhdfs_hdfsPread
+#define hdfsWrite libhdfs_hdfsWrite
+#define hdfsFlush libhdfs_hdfsFlush
+#define hdfsHFlush libhdfs_hdfsHFlush
+#define hdfsHSync libhdfs_hdfsHSync
+#define hdfsAvailable libhdfs_hdfsAvailable
+#define hdfsCopy libhdfs_hdfsCopy
+#define hdfsMove libhdfs_hdfsMove
+#define hdfsDelete libhdfs_hdfsDelete
+#define hdfsRename libhdfs_hdfsRename
+#define hdfsGetWorkingDirectory libhdfs_hdfsGetWorkingDirectory
+#define hdfsSetWorkingDirectory libhdfs_hdfsSetWorkingDirectory
+#define hdfsCreateDirectory libhdfs_hdfsCreateDirectory
+#define hdfsSetReplication libhdfs_hdfsSetReplication
+#define hdfsListDirectory libhdfs_hdfsListDirectory
+#define hdfsGetPathInfo libhdfs_hdfsGetPathInfo
+#define hdfsFreeFileInfo libhdfs_hdfsFreeFileInfo
+#define hdfsFileIsEncrypted libhdfs_hdfsFileIsEncrypted
+#define hdfsGetHosts libhdfs_hdfsGetHosts
+#define hdfsFreeHosts libhdfs_hdfsFreeHosts
+#define hdfsGetDefaultBlockSize libhdfs_hdfsGetDefaultBlockSize
+#define hdfsGetDefaultBlockSizeAtPath libhdfs_hdfsGetDefaultBlockSizeAtPath
+#define hdfsGetCapacity libhdfs_hdfsGetCapacity
+#define hdfsGetUsed libhdfs_hdfsGetUsed
+#define hdfsChown libhdfs_hdfsChown
+#define hdfsChmod libhdfs_hdfsChmod
+#define hdfsUtime libhdfs_hdfsUtime
+#define hadoopRzOptionsAlloc libhdfs_hadoopRzOptionsAlloc
+#define hadoopRzOptionsSetSkipChecksum libhdfs_hadoopRzOptionsSetSkipChecksum
+#define hadoopRzOptionsSetByteBufferPool libhdfs_hadoopRzOptionsSetByteBufferPool
+#define hadoopRzOptionsFree libhdfs_hadoopRzOptionsFree
+#define hadoopReadZero libhdfs_hadoopReadZero
+#define hadoopRzBufferLength libhdfs_hadoopRzBufferLength
+#define hadoopRzBufferGet libhdfs_hadoopRzBufferGet
+#define hadoopRzBufferFree libhdfs_hadoopRzBufferFree
+#define hdfs_internal libhdfs_hdfs_internal
+#define hdfsFS libhdfs_hdfsFS
+#define hdfsFile_internal libhdfs_hdfsFile_internal
+#define hdfsFile libhdfs_hdfsFile
+#define tObjectKind libhdfs_tObjectKind
+#define kObjectKindFile libhdfs_kObjectKindFile
+#define kObjectKindDirectory libhdfs_kObjectKindDirectory
+#define hdfsReadStatistics libhdfs_hdfsReadStatistics
+#define hdfsFileInfo libhdfs_hdfsFileInfo

+ 92 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_undefs.h

@@ -0,0 +1,92 @@
+/**
+ * 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.
+ */
+#undef hdfsFileIsOpenForRead
+#undef hdfsFileIsOpenForWrite
+#undef hdfsFileGetReadStatistics
+#undef hdfsReadStatisticsGetRemoteBytesRead
+#undef hdfsFileClearReadStatistics
+#undef hdfsFileFreeReadStatistics
+#undef hdfsConnectAsUser
+#undef hdfsConnect
+#undef hdfsConnectAsUserNewInstance
+#undef hdfsConnectNewInstance
+#undef hdfsBuilderConnect
+#undef hdfsNewBuilder
+#undef hdfsBuilderSetForceNewInstance
+#undef hdfsBuilderSetNameNode
+#undef hdfsBuilderSetNameNodePort
+#undef hdfsBuilderSetUserName
+#undef hdfsBuilderSetKerbTicketCachePath
+#undef hdfsFreeBuilder
+#undef hdfsBuilderConfSetStr
+#undef hdfsConfGetStr
+#undef hdfsConfGetInt
+#undef hdfsConfStrFree
+#undef hdfsDisconnect
+#undef hdfsOpenFile
+#undef hdfsTruncateFile
+#undef hdfsUnbufferFile
+#undef hdfsCloseFile
+#undef hdfsExists
+#undef hdfsSeek
+#undef hdfsTell
+#undef hdfsRead
+#undef hdfsPread
+#undef hdfsWrite
+#undef hdfsFlush
+#undef hdfsHFlush
+#undef hdfsHSync
+#undef hdfsAvailable
+#undef hdfsCopy
+#undef hdfsMove
+#undef hdfsDelete
+#undef hdfsRename
+#undef hdfsGetWorkingDirectory
+#undef hdfsSetWorkingDirectory
+#undef hdfsCreateDirectory
+#undef hdfsSetReplication
+#undef hdfsListDirectory
+#undef hdfsGetPathInfo
+#undef hdfsFreeFileInfo
+#undef hdfsFileIsEncrypted
+#undef hdfsGetHosts
+#undef hdfsFreeHosts
+#undef hdfsGetDefaultBlockSize
+#undef hdfsGetDefaultBlockSizeAtPath
+#undef hdfsGetCapacity
+#undef hdfsGetUsed
+#undef hdfsChown
+#undef hdfsChmod
+#undef hdfsUtime
+#undef hadoopRzOptionsAlloc
+#undef hadoopRzOptionsSetSkipChecksum
+#undef hadoopRzOptionsSetByteBufferPool
+#undef hadoopRzOptionsFree
+#undef hadoopReadZero
+#undef hadoopRzBufferLength
+#undef hadoopRzBufferGet
+#undef hadoopRzBufferFree
+#undef hdfs_internal
+#undef hdfsFS
+#undef hdfsFile_internal
+#undef hdfsFile
+#undef tObjectKind
+#undef kObjectKindFile
+#undef kObjectKindDirectory
+#undef hdfsReadStatistics
+#undef hdfsFileInfo

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfspp_wrapper.cc

@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+/* Rename libhdfspp structs and functions */
+#include "libhdfspp_wrapper_defines.h"
+#include "bindings/c/hdfs.cc"
+#include "libhdfs_wrapper_undefs.h"

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfspp_wrapper.h

@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+/* "Original" symbols can be included elsewhere. */
+#undef LIBHDFS_HDFS_H
+
+/* Rename libhdfspp structs and functions */
+#include "libhdfspp_wrapper_defines.h"
+#include "hdfs/hdfs.h"
+#include "libhdfs_wrapper_undefs.h"
+
+/* "Original" symbols can be included elsewhere. */
+#undef LIBHDFS_HDFS_H

+ 92 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfspp_wrapper_defines.h

@@ -0,0 +1,92 @@
+/**
+ * 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.
+ */
+#define hdfsFileIsOpenForRead libhdfspp_hdfsFileIsOpenForRead
+#define hdfsFileIsOpenForWrite libhdfspp_hdfsFileIsOpenForWrite
+#define hdfsFileGetReadStatistics libhdfspp_hdfsFileGetReadStatistics
+#define hdfsReadStatisticsGetRemoteBytesRead libhdfspp_hdfsReadStatisticsGetRemoteBytesRead
+#define hdfsFileClearReadStatistics libhdfspp_hdfsFileClearReadStatistics
+#define hdfsFileFreeReadStatistics libhdfspp_hdfsFileFreeReadStatistics
+#define hdfsConnectAsUser libhdfspp_hdfsConnectAsUser
+#define hdfsConnect libhdfspp_hdfsConnect
+#define hdfsConnectAsUserNewInstance libhdfspp_hdfsConnectAsUserNewInstance
+#define hdfsConnectNewInstance libhdfspp_hdfsConnectNewInstance
+#define hdfsBuilderConnect libhdfspp_hdfsBuilderConnect
+#define hdfsNewBuilder libhdfspp_hdfsNewBuilder
+#define hdfsBuilderSetForceNewInstance libhdfspp_hdfsBuilderSetForceNewInstance
+#define hdfsBuilderSetNameNode libhdfspp_hdfsBuilderSetNameNode
+#define hdfsBuilderSetNameNodePort libhdfspp_hdfsBuilderSetNameNodePort
+#define hdfsBuilderSetUserName libhdfspp_hdfsBuilderSetUserName
+#define hdfsBuilderSetKerbTicketCachePath libhdfspp_hdfsBuilderSetKerbTicketCachePath
+#define hdfsFreeBuilder libhdfspp_hdfsFreeBuilder
+#define hdfsBuilderConfSetStr libhdfspp_hdfsBuilderConfSetStr
+#define hdfsConfGetStr libhdfspp_hdfsConfGetStr
+#define hdfsConfGetInt libhdfspp_hdfsConfGetInt
+#define hdfsConfStrFree libhdfspp_hdfsConfStrFree
+#define hdfsDisconnect libhdfspp_hdfsDisconnect
+#define hdfsOpenFile libhdfspp_hdfsOpenFile
+#define hdfsTruncateFile libhdfspp_hdfsTruncateFile
+#define hdfsUnbufferFile libhdfspp_hdfsUnbufferFile
+#define hdfsCloseFile libhdfspp_hdfsCloseFile
+#define hdfsExists libhdfspp_hdfsExists
+#define hdfsSeek libhdfspp_hdfsSeek
+#define hdfsTell libhdfspp_hdfsTell
+#define hdfsRead libhdfspp_hdfsRead
+#define hdfsPread libhdfspp_hdfsPread
+#define hdfsWrite libhdfspp_hdfsWrite
+#define hdfsFlush libhdfspp_hdfsFlush
+#define hdfsHFlush libhdfspp_hdfsHFlush
+#define hdfsHSync libhdfspp_hdfsHSync
+#define hdfsAvailable libhdfspp_hdfsAvailable
+#define hdfsCopy libhdfspp_hdfsCopy
+#define hdfsMove libhdfspp_hdfsMove
+#define hdfsDelete libhdfspp_hdfsDelete
+#define hdfsRename libhdfspp_hdfsRename
+#define hdfsGetWorkingDirectory libhdfspp_hdfsGetWorkingDirectory
+#define hdfsSetWorkingDirectory libhdfspp_hdfsSetWorkingDirectory
+#define hdfsCreateDirectory libhdfspp_hdfsCreateDirectory
+#define hdfsSetReplication libhdfspp_hdfsSetReplication
+#define hdfsListDirectory libhdfspp_hdfsListDirectory
+#define hdfsGetPathInfo libhdfspp_hdfsGetPathInfo
+#define hdfsFreeFileInfo libhdfspp_hdfsFreeFileInfo
+#define hdfsFileIsEncrypted libhdfspp_hdfsFileIsEncrypted
+#define hdfsGetHosts libhdfspp_hdfsGetHosts
+#define hdfsFreeHosts libhdfspp_hdfsFreeHosts
+#define hdfsGetDefaultBlockSize libhdfspp_hdfsGetDefaultBlockSize
+#define hdfsGetDefaultBlockSizeAtPath libhdfspp_hdfsGetDefaultBlockSizeAtPath
+#define hdfsGetCapacity libhdfspp_hdfsGetCapacity
+#define hdfsGetUsed libhdfspp_hdfsGetUsed
+#define hdfsChown libhdfspp_hdfsChown
+#define hdfsChmod libhdfspp_hdfsChmod
+#define hdfsUtime libhdfspp_hdfsUtime
+#define hadoopRzOptionsAlloc libhdfspp_hadoopRzOptionsAlloc
+#define hadoopRzOptionsSetSkipChecksum libhdfspp_hadoopRzOptionsSetSkipChecksum
+#define hadoopRzOptionsSetByteBufferPool libhdfspp_hadoopRzOptionsSetByteBufferPool
+#define hadoopRzOptionsFree libhdfspp_hadoopRzOptionsFree
+#define hadoopReadZero libhdfspp_hadoopReadZero
+#define hadoopRzBufferLength libhdfspp_hadoopRzBufferLength
+#define hadoopRzBufferGet libhdfspp_hadoopRzBufferGet
+#define hadoopRzBufferFree libhdfspp_hadoopRzBufferFree
+#define hdfs_internal libhdfspp_hdfs_internal
+#define hdfsFS libhdfspp_hdfsFS
+#define hdfsFile_internal libhdfspp_hdfsFile_internal
+#define hdfsFile libhdfspp_hdfsFile
+#define tObjectKind libhdfspp_tObjectKind
+#define kObjectKindFile libhdfspp_kObjectKindFile
+#define kObjectKindDirectory libhdfspp_kObjectKindDirectory
+#define hdfsReadStatistics libhdfspp_hdfsReadStatistics
+#define hdfsFileInfo libhdfspp_hdfsFileInfo