ソースを参照

HDFS-573. Merging change r1616814 from trunk to branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1616817 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 10 年 前
コミット
97dd53a608
34 ファイル変更1935 行追加591 行削除
  1. 1 0
      BUILDING.txt
  2. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  3. 101 16
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  4. 57 21
      hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
  5. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt
  6. 271 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/common/htable.c
  7. 161 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/common/htable.h
  8. 33 32
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c
  9. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.h
  10. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.c
  11. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.h
  12. 194 160
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
  13. 90 175
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c
  14. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h
  15. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c
  16. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/mutexes.h
  17. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/mutexes.c
  18. 34 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/platform.h
  19. 52 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/thread.c
  20. 80 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/thread_local_storage.c
  21. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/thread.h
  22. 75 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/thread_local_storage.h
  23. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/inttypes.h
  24. 52 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/mutexes.c
  25. 86 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/platform.h
  26. 66 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c
  27. 164 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread_local_storage.c
  28. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/unistd.h
  29. 119 113
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_ops.c
  30. 11 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_read.c
  31. 17 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_write.c
  32. 11 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_zerocopy.c
  33. 18 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c
  34. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_native_mini_dfs.c

+ 1 - 0
BUILDING.txt

@@ -189,6 +189,7 @@ Requirements:
 * Maven 3.0 or later
 * Findbugs 1.3.9 (if running findbugs)
 * ProtocolBuffer 2.5.0
+* CMake 2.6 or newer
 * Windows SDK or Visual Studio 2010 Professional
 * Unix command-line tools from GnuWin32 or Cygwin: sh, mkdir, rm, cp, tar, gzip
 * zlib headers (if building native code bindings for zlib)

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -124,6 +124,8 @@ Release 2.6.0 - UNRELEASED
     HDFS-6772. Get DN storages out of blockContentsStale state faster after
     NN restarts. (Ming Ma via Arpit Agarwal)
 
+    HDFS-573. Porting libhdfs to Windows. (cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)

+ 101 - 16
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -503,16 +503,97 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
 
   <profiles>
     <profile>
-      <id>windows</id>
+      <id>native-win</id>
       <activation>
         <activeByDefault>false</activeByDefault>
         <os>
           <family>windows</family>
         </os>
       </activation>
-      <properties>
-        <windows.build>true</windows.build>
-      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-enforcer-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>enforce-os</id>
+                <goals>
+                  <goal>enforce</goal>
+                </goals>
+                <configuration>
+                  <rules>
+                    <requireOS>
+                      <family>windows</family>
+                      <message>native-win build only supported on Windows</message>
+                    </requireOS>
+                  </rules>
+                  <fail>true</fail>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>make</id>
+                <phase>compile</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                    <mkdir dir="${project.build.directory}/native"/>
+                    <exec executable="cmake" dir="${project.build.directory}/native"
+                        failonerror="true">
+                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model} -DREQUIRE_LIBWEBHDFS=${require.libwebhdfs} -DREQUIRE_FUSE=${require.fuse} -G 'Visual Studio 10 Win64'"/>
+                    </exec>
+                    <exec executable="msbuild" dir="${project.build.directory}/native"
+                        failonerror="true">
+                      <arg line="ALL_BUILD.vcxproj /nologo /p:Configuration=Release"/>
+                    </exec>
+                    <!-- Copy for inclusion in distribution. -->
+                    <copy todir="${project.build.directory}/bin">
+                      <fileset dir="${project.build.directory}/native/target/bin/Release"/>
+                    </copy>
+                  </target>
+                </configuration>
+              </execution>
+              <execution>
+                <id>native_tests</id>
+                <phase>test</phase>
+                <goals><goal>run</goal></goals>
+                <configuration>
+                  <skip>${skipTests}</skip>
+                  <target>
+                    <property name="compile_classpath" refid="maven.compile.classpath"/>
+                    <property name="test_classpath" refid="maven.test.classpath"/>
+                    <macrodef name="run-test">
+                      <attribute name="test"/>
+                      <sequential>
+                        <echo message="Running @{test}"/>
+                        <exec executable="${project.build.directory}/native/Release/@{test}" failonerror="true" dir="${project.build.directory}/native/">
+                          <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
+                          <!-- HADOOP_HOME required to find winutils. -->
+                          <env key="HADOOP_HOME" value="${hadoop.common.build.dir}"/>
+                          <!-- Make sure hadoop.dll and jvm.dll are on PATH. -->
+                          <env key="PATH" value="${env.PATH};${hadoop.common.build.dir}/bin;${java.home}/jre/bin/server;${java.home}/bin/server"/>
+                        </exec>
+                        <echo message="Finished @{test}"/>
+                      </sequential>
+                    </macrodef>
+                    <run-test test="test_libhdfs_threaded"/>
+                    <echo message="Skipping test_libhdfs_zerocopy"/>
+                    <run-test test="test_native_mini_dfs"/>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
     </profile>
     <profile>
       <id>native</id>
@@ -550,21 +631,25 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <phase>test</phase>
                 <goals><goal>run</goal></goals>
                 <configuration>
+                  <skip>${skipTests}</skip>
                   <target>
                     <property name="compile_classpath" refid="maven.compile.classpath"/>
                     <property name="test_classpath" refid="maven.test.classpath"/>
-                    <exec executable="sh" failonerror="true" dir="${project.build.directory}/native/">
-                      <arg value="-c"/>
-                      <arg value="[ x$SKIPTESTS = xtrue ] || ${project.build.directory}/native/test_libhdfs_threaded"/>
-                      <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
-                      <env key="SKIPTESTS" value="${skipTests}"/>
-                    </exec>
-                    <exec executable="sh" failonerror="true" dir="${project.build.directory}/native/">
-                        <arg value="-c"/>
-                        <arg value="[ x$SKIPTESTS = xtrue ] || ${project.build.directory}/native/test_native_mini_dfs"/>
-                      <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
-                      <env key="SKIPTESTS" value="${skipTests}"/>
-                    </exec>
+                    <macrodef name="run-test">
+                      <attribute name="test"/>
+                      <sequential>
+                        <echo message="Running @{test}"/>
+                        <exec executable="${project.build.directory}/native/@{test}" failonerror="true" dir="${project.build.directory}/native/">
+                          <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
+                          <!-- Make sure libhadoop.so is on LD_LIBRARY_PATH. -->
+                          <env key="LD_LIBRARY_PATH" value="${env.LD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
+                        </exec>
+                        <echo message="Finished @{test}"/>
+                      </sequential>
+                    </macrodef>
+                    <run-test test="test_libhdfs_threaded"/>
+                    <run-test test="test_libhdfs_zerocopy"/>
+                    <run-test test="test_native_mini_dfs"/>
                   </target>
                 </configuration>
               </execution>

+ 57 - 21
hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt

@@ -76,9 +76,39 @@ if (NOT GENERATED_JAVAH)
     MESSAGE(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
 endif (NOT GENERATED_JAVAH)
 
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
+if (WIN32)
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /O2")
+
+    # Set warning level 4.
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /W4")
+
+    # Skip "unreferenced formal parameter".
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /wd4100")
+
+    # Skip "conditional expression is constant".
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /wd4127")
+
+    # Skip deprecated POSIX function warnings.
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_CRT_NONSTDC_NO_DEPRECATE")
+
+    # Skip CRT non-secure function warnings.  If we can convert usage of
+    # strerror, getenv and ctime to their secure CRT equivalents, then we can
+    # re-enable the CRT non-secure function warnings.
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_CRT_SECURE_NO_WARNINGS")
+
+    # Omit unneeded headers.
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -DWIN32_LEAN_AND_MEAN")
+
+    set(OS_DIR main/native/libhdfs/os/windows)
+    set(OUT_DIR target/bin)
+else (WIN32)
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
+    set(OS_DIR main/native/libhdfs/os/posix)
+    set(OS_LINK_LIBRARIES pthread)
+    set(OUT_DIR target/usr/local/lib)
+endif (WIN32)
 
 include_directories(
     ${GENERATED_JAVAH}
@@ -87,6 +117,7 @@ include_directories(
     ${JNI_INCLUDE_DIRS}
     main/native
     main/native/libhdfs
+    ${OS_DIR}
 )
 
 set(_FUSE_DFS_VERSION 0.1.0)
@@ -96,6 +127,9 @@ add_dual_library(hdfs
     main/native/libhdfs/exception.c
     main/native/libhdfs/jni_helper.c
     main/native/libhdfs/hdfs.c
+    main/native/libhdfs/common/htable.c
+    ${OS_DIR}/mutexes.c
+    ${OS_DIR}/thread_local_storage.c
 )
 if (NEED_LINK_DL)
    set(LIB_DL dl)
@@ -104,17 +138,14 @@ endif(NEED_LINK_DL)
 target_link_dual_libraries(hdfs
     ${JAVA_JVM_LIBRARY}
     ${LIB_DL}
-    pthread
+    ${OS_LINK_LIBRARIES}
 )
-dual_output_directory(hdfs target/usr/local/lib)
+
+dual_output_directory(hdfs ${OUT_DIR})
 set(LIBHDFS_VERSION "0.0.0")
 set_target_properties(hdfs PROPERTIES
     SOVERSION ${LIBHDFS_VERSION})
 
-add_library(posix_util
-    main/native/util/posix_util.c
-)
-
 add_executable(test_libhdfs_ops
     main/native/libhdfs/test/test_libhdfs_ops.c
 )
@@ -156,11 +187,12 @@ target_link_libraries(test_native_mini_dfs
 add_executable(test_libhdfs_threaded
     main/native/libhdfs/expect.c
     main/native/libhdfs/test_libhdfs_threaded.c
+    ${OS_DIR}/thread.c
 )
 target_link_libraries(test_libhdfs_threaded
     hdfs
     native_mini_dfs
-    pthread
+    ${OS_LINK_LIBRARIES}
 )
 
 add_executable(test_libhdfs_zerocopy
@@ -170,17 +202,21 @@ add_executable(test_libhdfs_zerocopy
 target_link_libraries(test_libhdfs_zerocopy
     hdfs
     native_mini_dfs
-    pthread
-)
-
-add_executable(test_libhdfs_vecsum
-    main/native/libhdfs/test/vecsum.c
-)
-target_link_libraries(test_libhdfs_vecsum
-    hdfs
-    pthread
-    rt
-)
+    ${OS_LINK_LIBRARIES}
+)
+
+# Skip vecsum on Windows.  This could be made to work in the future by
+# introducing an abstraction layer over the sys/mman.h functions.
+if (NOT WIN32)
+    add_executable(test_libhdfs_vecsum
+        main/native/libhdfs/test/vecsum.c
+    )
+    target_link_libraries(test_libhdfs_vecsum
+        hdfs
+        pthread
+        rt
+    )
+endif(NOT WIN32)
 
 IF(REQUIRE_LIBWEBHDFS)
     add_subdirectory(contrib/libwebhdfs)

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt

@@ -37,6 +37,10 @@ ELSE (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
 ENDIF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
 
 IF(FUSE_FOUND)
+    add_library(posix_util
+        ../util/posix_util.c
+    )
+
     add_executable(fuse_dfs
         fuse_dfs.c
         fuse_options.c 

+ 271 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/common/htable.c

@@ -0,0 +1,271 @@
+/**
+ * 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 "common/htable.h"
+
+#include <errno.h>
+#include <inttypes.h>
+#include <stdlib.h>
+#include <string.h>
+#include <unistd.h>
+
+struct htable_pair {
+    void *key;
+    void *val;
+};
+
+/**
+ * A hash table which uses linear probing.
+ */
+struct htable {
+    uint32_t capacity;
+    uint32_t used;
+    htable_hash_fn_t hash_fun;
+    htable_eq_fn_t eq_fun;
+    struct htable_pair *elem;
+};
+
+/**
+ * An internal function for inserting a value into the hash table.
+ *
+ * Note: this function assumes that you have made enough space in the table.
+ *
+ * @param nelem         The new element to insert.
+ * @param capacity      The capacity of the hash table.
+ * @param hash_fun      The hash function to use.
+ * @param key           The key to insert.
+ * @param val           The value to insert.
+ */
+static void htable_insert_internal(struct htable_pair *nelem, 
+        uint32_t capacity, htable_hash_fn_t hash_fun, void *key,
+        void *val)
+{
+    uint32_t i;
+
+    i = hash_fun(key, capacity);
+    while (1) {
+        if (!nelem[i].key) {
+            nelem[i].key = key;
+            nelem[i].val = val;
+            return;
+        }
+        i++;
+        if (i == capacity) {
+            i = 0;
+        }
+    }
+}
+
+static int htable_realloc(struct htable *htable, uint32_t new_capacity)
+{
+    struct htable_pair *nelem;
+    uint32_t i, old_capacity = htable->capacity;
+    htable_hash_fn_t hash_fun = htable->hash_fun;
+
+    nelem = calloc(new_capacity, sizeof(struct htable_pair));
+    if (!nelem) {
+        return ENOMEM;
+    }
+    for (i = 0; i < old_capacity; i++) {
+        struct htable_pair *pair = htable->elem + i;
+        htable_insert_internal(nelem, new_capacity, hash_fun,
+                               pair->key, pair->val);
+    }
+    free(htable->elem);
+    htable->elem = nelem;
+    htable->capacity = new_capacity;
+    return 0;
+}
+
+struct htable *htable_alloc(uint32_t size,
+                htable_hash_fn_t hash_fun, htable_eq_fn_t eq_fun)
+{
+    struct htable *htable;
+
+    htable = calloc(1, sizeof(*htable));
+    if (!htable) {
+        return NULL;
+    }
+    size = (size + 1) >> 1;
+    size = size << 1;
+    if (size < HTABLE_MIN_SIZE) {
+        size = HTABLE_MIN_SIZE;
+    }
+    htable->hash_fun = hash_fun;
+    htable->eq_fun = eq_fun;
+    htable->used = 0;
+    if (htable_realloc(htable, size)) {
+        free(htable);
+        return NULL;
+    }
+    return htable;
+}
+
+void htable_visit(struct htable *htable, visitor_fn_t fun, void *ctx)
+{
+    uint32_t i;
+
+    for (i = 0; i != htable->capacity; ++i) {
+        struct htable_pair *elem = htable->elem + i;
+        if (elem->key) {
+            fun(ctx, elem->key, elem->val);
+        }
+    }
+}
+
+void htable_free(struct htable *htable)
+{
+    if (htable) {
+        free(htable->elem);
+        free(htable);
+    }
+}
+
+int htable_put(struct htable *htable, void *key, void *val)
+{
+    int ret;
+    uint32_t nused;
+
+    // NULL is not a valid key value.
+    // This helps us implement htable_get_internal efficiently, since we know
+    // that we can stop when we encounter the first NULL key.
+    if (!key) {
+        return EINVAL;
+    }
+    // NULL is not a valid value.  Otherwise the results of htable_get would
+    // be confusing (does a NULL return mean entry not found, or that the
+    // entry was found and was NULL?) 
+    if (!val) {
+        return EINVAL;
+    }
+    // Re-hash if we have used more than half of the hash table
+    nused = htable->used + 1;
+    if (nused >= (htable->capacity / 2)) {
+        ret = htable_realloc(htable, htable->capacity * 2);
+        if (ret)
+            return ret;
+    }
+    htable_insert_internal(htable->elem, htable->capacity,
+                                htable->hash_fun, key, val);
+    htable->used++;
+    return 0;
+}
+
+static int htable_get_internal(const struct htable *htable,
+                               const void *key, uint32_t *out)
+{
+    uint32_t start_idx, idx;
+
+    start_idx = htable->hash_fun(key, htable->capacity);
+    idx = start_idx;
+    while (1) {
+        struct htable_pair *pair = htable->elem + idx;
+        if (!pair->key) {
+            // We always maintain the invariant that the entries corresponding
+            // to a given key are stored in a contiguous block, not separated
+            // by any NULLs.  So if we encounter a NULL, our search is over.
+            return ENOENT;
+        } else if (htable->eq_fun(pair->key, key)) {
+            *out = idx;
+            return 0;
+        }
+        idx++;
+        if (idx == htable->capacity) {
+            idx = 0;
+        }
+        if (idx == start_idx) {
+            return ENOENT;
+        }
+    }
+}
+
+void *htable_get(const struct htable *htable, const void *key)
+{
+    uint32_t idx;
+
+    if (htable_get_internal(htable, key, &idx)) {
+        return NULL;
+    }
+    return htable->elem[idx].val;
+}
+
+void htable_pop(struct htable *htable, const void *key,
+                void **found_key, void **found_val)
+{
+    uint32_t hole, i;
+    const void *nkey;
+
+    if (htable_get_internal(htable, key, &hole)) {
+        *found_key = NULL;
+        *found_val = NULL;
+        return;
+    }
+    i = hole;
+    htable->used--;
+    // We need to maintain the compactness invariant used in
+    // htable_get_internal.  This invariant specifies that the entries for any
+    // given key are never separated by NULLs (although they may be separated
+    // by entries for other keys.)
+    while (1) {
+        i++;
+        if (i == htable->capacity) {
+            i = 0;
+        }
+        nkey = htable->elem[i].key;
+        if (!nkey) {
+            *found_key = htable->elem[hole].key;
+            *found_val = htable->elem[hole].val;
+            htable->elem[hole].key = NULL;
+            htable->elem[hole].val = NULL;
+            return;
+        } else if (htable->eq_fun(key, nkey)) {
+            htable->elem[hole].key = htable->elem[i].key;
+            htable->elem[hole].val = htable->elem[i].val;
+            hole = i;
+        }
+    }
+}
+
+uint32_t htable_used(const struct htable *htable)
+{
+    return htable->used;
+}
+
+uint32_t htable_capacity(const struct htable *htable)
+{
+    return htable->capacity;
+}
+
+uint32_t ht_hash_string(const void *str, uint32_t max)
+{
+    const char *s = str;
+    uint32_t hash = 0;
+
+    while (*s) {
+        hash = (hash * 31) + *s;
+        s++;
+    }
+    return hash % max;
+}
+
+int ht_compare_string(const void *a, const void *b)
+{
+    return strcmp(a, b) == 0;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 161 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/common/htable.h

@@ -0,0 +1,161 @@
+/**
+ * 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 HADOOP_CORE_COMMON_HASH_TABLE
+#define HADOOP_CORE_COMMON_HASH_TABLE
+
+#include <inttypes.h>
+#include <stdio.h>
+#include <stdint.h>
+
+#define HTABLE_MIN_SIZE 4
+
+struct htable;
+
+/**
+ * An HTable hash function.
+ *
+ * @param key       The key.
+ * @param capacity  The total capacity.
+ *
+ * @return          The hash slot.  Must be less than the capacity.
+ */
+typedef uint32_t (*htable_hash_fn_t)(const void *key, uint32_t capacity);
+
+/**
+ * An HTable equality function.  Compares two keys.
+ *
+ * @param a         First key.
+ * @param b         Second key.
+ *
+ * @return          nonzero if the keys are equal.
+ */
+typedef int (*htable_eq_fn_t)(const void *a, const void *b);
+
+/**
+ * Allocate a new hash table.
+ *
+ * @param capacity  The minimum suggested starting capacity.
+ * @param hash_fun  The hash function to use in this hash table.
+ * @param eq_fun    The equals function to use in this hash table.
+ *
+ * @return          The new hash table on success; NULL on OOM.
+ */
+struct htable *htable_alloc(uint32_t capacity, htable_hash_fn_t hash_fun,
+                            htable_eq_fn_t eq_fun);
+
+typedef void (*visitor_fn_t)(void *ctx, void *key, void *val);
+
+/**
+ * Visit all of the entries in the hash table.
+ *
+ * @param htable    The hash table.
+ * @param fun       The callback function to invoke on each key and value.
+ * @param ctx       Context pointer to pass to the callback.
+ */
+void htable_visit(struct htable *htable, visitor_fn_t fun, void *ctx);
+
+/**
+ * Free the hash table.
+ *
+ * It is up the calling code to ensure that the keys and values inside the
+ * table are de-allocated, if that is necessary.
+ *
+ * @param htable    The hash table.
+ */
+void htable_free(struct htable *htable);
+
+/**
+ * Add an entry to the hash table.
+ *
+ * @param htable    The hash table.
+ * @param key       The key to add.  This cannot be NULL.
+ * @param fun       The value to add.  This cannot be NULL.
+ *
+ * @return          0 on success;
+ *                  EEXIST if the value already exists in the table;
+ *                  ENOMEM if there is not enough memory to add the element.
+ *                  EFBIG if the hash table has too many entries to fit in 32
+ *                      bits.
+ */
+int htable_put(struct htable *htable, void *key, void *val);
+
+/**
+ * Get an entry from the hash table.
+ *
+ * @param htable    The hash table.
+ * @param key       The key to find.
+ *
+ * @return          NULL if there is no such entry; the entry otherwise.
+ */
+void *htable_get(const struct htable *htable, const void *key);
+
+/**
+ * Get an entry from the hash table and remove it.
+ *
+ * @param htable    The hash table.
+ * @param key       The key for the entry find and remove.
+ * @param found_key (out param) NULL if the entry was not found; the found key
+ *                      otherwise.
+ * @param found_val (out param) NULL if the entry was not found; the found
+ *                      value otherwise.
+ */
+void htable_pop(struct htable *htable, const void *key,
+                void **found_key, void **found_val);
+
+/**
+ * Get the number of entries used in the hash table.
+ *
+ * @param htable    The hash table.
+ *
+ * @return          The number of entries used in the hash table.
+ */
+uint32_t htable_used(const struct htable *htable);
+
+/**
+ * Get the capacity of the hash table.
+ *
+ * @param htable    The hash table.
+ *
+ * @return          The capacity of the hash table.
+ */
+uint32_t htable_capacity(const struct htable *htable);
+
+/**
+ * Hash a string.
+ *
+ * @param str       The string.
+ * @param max       Maximum hash value
+ *
+ * @return          A number less than max.
+ */
+uint32_t ht_hash_string(const void *str, uint32_t max);
+
+/**
+ * Compare two strings.
+ *
+ * @param a         The first string.
+ * @param b         The second string.
+ *
+ * @return          1 if the strings are identical; 0 otherwise.
+ */
+int ht_compare_string(const void *a, const void *b);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 33 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c

@@ -19,8 +19,8 @@
 #include "exception.h"
 #include "hdfs.h"
 #include "jni_helper.h"
+#include "platform.h"
 
-#include <inttypes.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
@@ -35,54 +35,54 @@ struct ExceptionInfo {
 
 static const struct ExceptionInfo gExceptionInfo[] = {
     {
-        .name = "java.io.FileNotFoundException",
-        .noPrintFlag = NOPRINT_EXC_FILE_NOT_FOUND,
-        .excErrno = ENOENT,
+        "java.io.FileNotFoundException",
+        NOPRINT_EXC_FILE_NOT_FOUND,
+        ENOENT,
     },
     {
-        .name = "org.apache.hadoop.security.AccessControlException",
-        .noPrintFlag = NOPRINT_EXC_ACCESS_CONTROL,
-        .excErrno = EACCES,
+        "org.apache.hadoop.security.AccessControlException",
+        NOPRINT_EXC_ACCESS_CONTROL,
+        EACCES,
     },
     {
-        .name = "org.apache.hadoop.fs.UnresolvedLinkException",
-        .noPrintFlag = NOPRINT_EXC_UNRESOLVED_LINK,
-        .excErrno = ENOLINK,
+        "org.apache.hadoop.fs.UnresolvedLinkException",
+        NOPRINT_EXC_UNRESOLVED_LINK,
+        ENOLINK,
     },
     {
-        .name = "org.apache.hadoop.fs.ParentNotDirectoryException",
-        .noPrintFlag = NOPRINT_EXC_PARENT_NOT_DIRECTORY,
-        .excErrno = ENOTDIR,
+        "org.apache.hadoop.fs.ParentNotDirectoryException",
+        NOPRINT_EXC_PARENT_NOT_DIRECTORY,
+        ENOTDIR,
     },
     {
-        .name = "java.lang.IllegalArgumentException",
-        .noPrintFlag = NOPRINT_EXC_ILLEGAL_ARGUMENT,
-        .excErrno = EINVAL,
+        "java.lang.IllegalArgumentException",
+        NOPRINT_EXC_ILLEGAL_ARGUMENT,
+        EINVAL,
     },
     {
-        .name = "java.lang.OutOfMemoryError",
-        .noPrintFlag = 0,
-        .excErrno = ENOMEM,
+        "java.lang.OutOfMemoryError",
+        0,
+        ENOMEM,
     },
     {
-        .name = "org.apache.hadoop.hdfs.server.namenode.SafeModeException",
-        .noPrintFlag = 0,
-        .excErrno = EROFS,
+        "org.apache.hadoop.hdfs.server.namenode.SafeModeException",
+        0,
+        EROFS,
     },
     {
-        .name = "org.apache.hadoop.fs.FileAlreadyExistsException",
-        .noPrintFlag = 0,
-        .excErrno = EEXIST,
+        "org.apache.hadoop.fs.FileAlreadyExistsException",
+        0,
+        EEXIST,
     },
     {
-        .name = "org.apache.hadoop.hdfs.protocol.QuotaExceededException",
-        .noPrintFlag = 0,
-        .excErrno = EDQUOT,
+        "org.apache.hadoop.hdfs.protocol.QuotaExceededException",
+        0,
+        EDQUOT,
     },
     {
-        .name = "org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException",
-        .noPrintFlag = 0,
-        .excErrno = ESTALE,
+        "org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException",
+        0,
+        ESTALE,
     },
 };
 
@@ -113,6 +113,7 @@ int printExceptionAndFreeV(JNIEnv *env, jthrowable exc, int noPrintFlags,
     jstring jStr = NULL;
     jvalue jVal;
     jthrowable jthr;
+    const char *stackTrace;
 
     jthr = classNameOfObject(exc, env, &className);
     if (jthr) {
@@ -148,7 +149,7 @@ int printExceptionAndFreeV(JNIEnv *env, jthrowable exc, int noPrintFlags,
             destroyLocalReference(env, jthr);
         } else {
             jStr = jVal.l;
-            const char *stackTrace = (*env)->GetStringUTFChars(env, jStr, NULL);
+            stackTrace = (*env)->GetStringUTFChars(env, jStr, NULL);
             if (!stackTrace) {
                 fprintf(stderr, "(unable to get stack trace for %s exception: "
                         "GetStringUTFChars error.)\n", className);

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.h

@@ -34,13 +34,14 @@
  * usually not what you want.)
  */
 
+#include "platform.h"
+
 #include <jni.h>
 #include <stdio.h>
 
 #include <stdlib.h>
 #include <stdarg.h>
 #include <search.h>
-#include <pthread.h>
 #include <errno.h>
 
 /**
@@ -109,7 +110,7 @@ int printExceptionAndFreeV(JNIEnv *env, jthrowable exc, int noPrintFlags,
  *                        object.
  */
 int printExceptionAndFree(JNIEnv *env, jthrowable exc, int noPrintFlags,
-        const char *fmt, ...) __attribute__((format(printf, 4, 5)));  
+        const char *fmt, ...) TYPE_CHECKED_PRINTF_FORMAT(4, 5);
 
 /**
  * Print out information about the pending exception and free it.
@@ -124,7 +125,7 @@ int printExceptionAndFree(JNIEnv *env, jthrowable exc, int noPrintFlags,
  *                        object.
  */
 int printPendingExceptionAndFree(JNIEnv *env, int noPrintFlags,
-        const char *fmt, ...) __attribute__((format(printf, 3, 4)));  
+        const char *fmt, ...) TYPE_CHECKED_PRINTF_FORMAT(3, 4);
 
 /**
  * Get a local reference to the pending exception and clear it.
@@ -150,6 +151,7 @@ jthrowable getPendingExceptionAndClear(JNIEnv *env);
  * @return                A local reference to a RuntimeError
  */
 jthrowable newRuntimeError(JNIEnv *env, const char *fmt, ...)
-        __attribute__((format(printf, 2, 3)));
+        TYPE_CHECKED_PRINTF_FORMAT(2, 3);
 
+#undef TYPE_CHECKED_PRINTF_FORMAT
 #endif

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.c

@@ -49,18 +49,18 @@ int expectFileStats(hdfsFile file,
             stats->totalShortCircuitBytesRead,
             stats->totalZeroCopyBytesRead);
     if (expectedTotalBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalBytesRead, stats->totalBytesRead);
+        EXPECT_UINT64_EQ(expectedTotalBytesRead, stats->totalBytesRead);
     }
     if (expectedTotalLocalBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalLocalBytesRead,
+        EXPECT_UINT64_EQ(expectedTotalLocalBytesRead,
                       stats->totalLocalBytesRead);
     }
     if (expectedTotalShortCircuitBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalShortCircuitBytesRead,
+        EXPECT_UINT64_EQ(expectedTotalShortCircuitBytesRead,
                       stats->totalShortCircuitBytesRead);
     }
     if (expectedTotalZeroCopyBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalZeroCopyBytesRead,
+        EXPECT_UINT64_EQ(expectedTotalZeroCopyBytesRead,
                       stats->totalZeroCopyBytesRead);
     }
     hdfsFileFreeReadStatistics(stats);

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.h

@@ -126,6 +126,18 @@ struct hdfsFile_internal;
         } \
     } while (0);
 
+#define EXPECT_UINT64_EQ(x, y) \
+    do { \
+        uint64_t __my_ret__ = y; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ != (x)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "value %"PRIu64" (errno: %d): expected %"PRIu64"\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (x)); \
+            return -1; \
+        } \
+    } while (0);
+
 #define RETRY_ON_EINTR_GET_ERRNO(ret, expr) do { \
     ret = expr; \
     if (!ret) \

ファイルの差分が大きいため隠しています
+ 194 - 160
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c


+ 90 - 175
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c

@@ -19,20 +19,18 @@
 #include "config.h"
 #include "exception.h"
 #include "jni_helper.h"
+#include "platform.h"
+#include "common/htable.h"
+#include "os/mutexes.h"
+#include "os/thread_local_storage.h"
 
 #include <stdio.h> 
 #include <string.h> 
 
-static pthread_mutex_t hdfsHashMutex = PTHREAD_MUTEX_INITIALIZER;
-static pthread_mutex_t jvmMutex = PTHREAD_MUTEX_INITIALIZER;
-static volatile int hashTableInited = 0;
-
-#define LOCK_HASH_TABLE() pthread_mutex_lock(&hdfsHashMutex)
-#define UNLOCK_HASH_TABLE() pthread_mutex_unlock(&hdfsHashMutex)
-
+static struct htable *gClassRefHTable = NULL;
 
 /** The Native return types that methods could return */
-#define VOID          'V'
+#define JVOID         'V'
 #define JOBJECT       'L'
 #define JARRAYOBJECT  '['
 #define JBOOLEAN      'Z'
@@ -51,40 +49,10 @@ static volatile int hashTableInited = 0;
  */
 #define MAX_HASH_TABLE_ELEM 4096
 
-/** Key that allows us to retrieve thread-local storage */
-static pthread_key_t gTlsKey;
-
-/** nonzero if we succeeded in initializing gTlsKey. Protected by the jvmMutex */
-static int gTlsKeyInitialized = 0;
-
-/** Pthreads thread-local storage for each library thread. */
-struct hdfsTls {
-    JNIEnv *env;
-};
-
 /**
- * The function that is called whenever a thread with libhdfs thread local data
- * is destroyed.
- *
- * @param v         The thread-local data
+ * Length of buffer for retrieving created JVMs.  (We only ever create one.)
  */
-static void hdfsThreadDestructor(void *v)
-{
-    struct hdfsTls *tls = v;
-    JavaVM *vm;
-    JNIEnv *env = tls->env;
-    jint ret;
-
-    ret = (*env)->GetJavaVM(env, &vm);
-    if (ret) {
-        fprintf(stderr, "hdfsThreadDestructor: GetJavaVM failed with "
-                "error %d\n", ret);
-        (*env)->ExceptionDescribe(env);
-    } else {
-        (*vm)->DetachCurrentThread(vm);
-    }
-    free(tls);
-}
+#define VM_BUF_LENGTH 1
 
 void destroyLocalReference(JNIEnv *env, jobject jObject)
 {
@@ -138,67 +106,6 @@ jthrowable newCStr(JNIEnv *env, jstring jstr, char **out)
     return NULL;
 }
 
-static int hashTableInit(void)
-{
-    if (!hashTableInited) {
-        LOCK_HASH_TABLE();
-        if (!hashTableInited) {
-            if (hcreate(MAX_HASH_TABLE_ELEM) == 0) {
-                fprintf(stderr, "error creating hashtable, <%d>: %s\n",
-                        errno, strerror(errno));
-                UNLOCK_HASH_TABLE();
-                return 0;
-            } 
-            hashTableInited = 1;
-        }
-        UNLOCK_HASH_TABLE();
-    }
-    return 1;
-}
-
-
-static int insertEntryIntoTable(const char *key, void *data)
-{
-    ENTRY e, *ep;
-    if (key == NULL || data == NULL) {
-        return 0;
-    }
-    if (! hashTableInit()) {
-      return -1;
-    }
-    e.data = data;
-    e.key = (char*)key;
-    LOCK_HASH_TABLE();
-    ep = hsearch(e, ENTER);
-    UNLOCK_HASH_TABLE();
-    if (ep == NULL) {
-        fprintf(stderr, "warn adding key (%s) to hash table, <%d>: %s\n",
-                key, errno, strerror(errno));
-    }  
-    return 0;
-}
-
-
-
-static void* searchEntryFromTable(const char *key)
-{
-    ENTRY e,*ep;
-    if (key == NULL) {
-        return NULL;
-    }
-    hashTableInit();
-    e.key = (char*)key;
-    LOCK_HASH_TABLE();
-    ep = hsearch(e, FIND);
-    UNLOCK_HASH_TABLE();
-    if (ep != NULL) {
-        return ep->data;
-    }
-    return NULL;
-}
-
-
-
 jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
                  jobject instObj, const char *className,
                  const char *methName, const char *methSignature, ...)
@@ -235,7 +142,7 @@ jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
         }
         retval->l = jobj;
     }
-    else if (returnType == VOID) {
+    else if (returnType == JVOID) {
         if (methType == STATIC) {
             (*env)->CallStaticVoidMethodV(env, cls, mid, args);
         }
@@ -325,11 +232,11 @@ jthrowable methodIdFromClass(const char *className, const char *methName,
 {
     jclass cls;
     jthrowable jthr;
+    jmethodID mid = 0;
 
     jthr = globalClassReference(className, env, &cls);
     if (jthr)
         return jthr;
-    jmethodID mid = 0;
     jthr = validateMethodType(env, methType);
     if (jthr)
         return jthr;
@@ -350,25 +257,50 @@ jthrowable methodIdFromClass(const char *className, const char *methName,
 
 jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out)
 {
-    jclass clsLocalRef;
-    jclass cls = searchEntryFromTable(className);
-    if (cls) {
-        *out = cls;
-        return NULL;
+    jthrowable jthr = NULL;
+    jclass local_clazz = NULL;
+    jclass clazz = NULL;
+    int ret;
+
+    mutexLock(&hdfsHashMutex);
+    if (!gClassRefHTable) {
+        gClassRefHTable = htable_alloc(MAX_HASH_TABLE_ELEM, ht_hash_string,
+            ht_compare_string);
+        if (!gClassRefHTable) {
+            jthr = newRuntimeError(env, "htable_alloc failed\n");
+            goto done;
+        }
     }
-    clsLocalRef = (*env)->FindClass(env,className);
-    if (clsLocalRef == NULL) {
-        return getPendingExceptionAndClear(env);
+    clazz = htable_get(gClassRefHTable, className);
+    if (clazz) {
+        *out = clazz;
+        goto done;
     }
-    cls = (*env)->NewGlobalRef(env, clsLocalRef);
-    if (cls == NULL) {
-        (*env)->DeleteLocalRef(env, clsLocalRef);
-        return getPendingExceptionAndClear(env);
+    local_clazz = (*env)->FindClass(env,className);
+    if (!local_clazz) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
     }
-    (*env)->DeleteLocalRef(env, clsLocalRef);
-    insertEntryIntoTable(className, cls);
-    *out = cls;
-    return NULL;
+    clazz = (*env)->NewGlobalRef(env, local_clazz);
+    if (!clazz) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    ret = htable_put(gClassRefHTable, (void*)className, clazz);
+    if (ret) {
+        jthr = newRuntimeError(env, "htable_put failed with error "
+                               "code %d\n", ret);
+        goto done;
+    }
+    *out = clazz;
+    jthr = NULL;
+done:
+    mutexUnlock(&hdfsHashMutex);
+    (*env)->DeleteLocalRef(env, local_clazz);
+    if (jthr && clazz) {
+        (*env)->DeleteGlobalRef(env, clazz);
+    }
+    return jthr;
 }
 
 jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name)
@@ -436,14 +368,24 @@ done:
  */
 static JNIEnv* getGlobalJNIEnv(void)
 {
-    const jsize vmBufLength = 1;
-    JavaVM* vmBuf[vmBufLength]; 
+    JavaVM* vmBuf[VM_BUF_LENGTH]; 
     JNIEnv *env;
     jint rv = 0; 
     jint noVMs = 0;
     jthrowable jthr;
+    char *hadoopClassPath;
+    const char *hadoopClassPathVMArg = "-Djava.class.path=";
+    size_t optHadoopClassPathLen;
+    char *optHadoopClassPath;
+    int noArgs = 1;
+    char *hadoopJvmArgs;
+    char jvmArgDelims[] = " ";
+    char *str, *token, *savePtr;
+    JavaVMInitArgs vm_args;
+    JavaVM *vm;
+    JavaVMOption *options;
 
-    rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), vmBufLength, &noVMs);
+    rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), VM_BUF_LENGTH, &noVMs);
     if (rv != 0) {
         fprintf(stderr, "JNI_GetCreatedJavaVMs failed with error: %d\n", rv);
         return NULL;
@@ -451,23 +393,19 @@ static JNIEnv* getGlobalJNIEnv(void)
 
     if (noVMs == 0) {
         //Get the environment variables for initializing the JVM
-        char *hadoopClassPath = getenv("CLASSPATH");
+        hadoopClassPath = getenv("CLASSPATH");
         if (hadoopClassPath == NULL) {
             fprintf(stderr, "Environment variable CLASSPATH not set!\n");
             return NULL;
         } 
-        char *hadoopClassPathVMArg = "-Djava.class.path=";
-        size_t optHadoopClassPathLen = strlen(hadoopClassPath) + 
+        optHadoopClassPathLen = strlen(hadoopClassPath) + 
           strlen(hadoopClassPathVMArg) + 1;
-        char *optHadoopClassPath = malloc(sizeof(char)*optHadoopClassPathLen);
+        optHadoopClassPath = malloc(sizeof(char)*optHadoopClassPathLen);
         snprintf(optHadoopClassPath, optHadoopClassPathLen,
                 "%s%s", hadoopClassPathVMArg, hadoopClassPath);
 
         // Determine the # of LIBHDFS_OPTS args
-        int noArgs = 1;
-        char *hadoopJvmArgs = getenv("LIBHDFS_OPTS");
-        char jvmArgDelims[] = " ";
-        char *str, *token, *savePtr;
+        hadoopJvmArgs = getenv("LIBHDFS_OPTS");
         if (hadoopJvmArgs != NULL)  {
           hadoopJvmArgs = strdup(hadoopJvmArgs);
           for (noArgs = 1, str = hadoopJvmArgs; ; noArgs++, str = NULL) {
@@ -480,7 +418,12 @@ static JNIEnv* getGlobalJNIEnv(void)
         }
 
         // Now that we know the # args, populate the options array
-        JavaVMOption options[noArgs];
+        options = calloc(noArgs, sizeof(JavaVMOption));
+        if (!options) {
+          fputs("Call to calloc failed\n", stderr);
+          free(optHadoopClassPath);
+          return NULL;
+        }
         options[0].optionString = optHadoopClassPath;
         hadoopJvmArgs = getenv("LIBHDFS_OPTS");
 	if (hadoopJvmArgs != NULL)  {
@@ -495,8 +438,6 @@ static JNIEnv* getGlobalJNIEnv(void)
         }
 
         //Create the VM
-        JavaVMInitArgs vm_args;
-        JavaVM *vm;
         vm_args.version = JNI_VERSION_1_2;
         vm_args.options = options;
         vm_args.nOptions = noArgs; 
@@ -508,6 +449,7 @@ static JNIEnv* getGlobalJNIEnv(void)
           free(hadoopJvmArgs);
         }
         free(optHadoopClassPath);
+        free(options);
 
         if (rv != 0) {
             fprintf(stderr, "Call to JNI_CreateJavaVM failed "
@@ -523,7 +465,7 @@ static JNIEnv* getGlobalJNIEnv(void)
     }
     else {
         //Attach this thread to the VM
-        JavaVM* vm = vmBuf[0];
+        vm = vmBuf[0];
         rv = (*vm)->AttachCurrentThread(vm, (void*)&env, 0);
         if (rv != 0) {
             fprintf(stderr, "Call to AttachCurrentThread "
@@ -557,54 +499,27 @@ static JNIEnv* getGlobalJNIEnv(void)
 JNIEnv* getJNIEnv(void)
 {
     JNIEnv *env;
-    struct hdfsTls *tls;
-    int ret;
-
-#ifdef HAVE_BETTER_TLS
-    static __thread struct hdfsTls *quickTls = NULL;
-    if (quickTls)
-        return quickTls->env;
-#endif
-    pthread_mutex_lock(&jvmMutex);
-    if (!gTlsKeyInitialized) {
-        ret = pthread_key_create(&gTlsKey, hdfsThreadDestructor);
-        if (ret) {
-            pthread_mutex_unlock(&jvmMutex);
-            fprintf(stderr, "getJNIEnv: pthread_key_create failed with "
-                "error %d\n", ret);
-            return NULL;
-        }
-        gTlsKeyInitialized = 1;
+    THREAD_LOCAL_STORAGE_GET_QUICK();
+    mutexLock(&jvmMutex);
+    if (threadLocalStorageGet(&env)) {
+      mutexUnlock(&jvmMutex);
+      return NULL;
     }
-    tls = pthread_getspecific(gTlsKey);
-    if (tls) {
-        pthread_mutex_unlock(&jvmMutex);
-        return tls->env;
+    if (env) {
+      mutexUnlock(&jvmMutex);
+      return env;
     }
 
     env = getGlobalJNIEnv();
-    pthread_mutex_unlock(&jvmMutex);
+    mutexUnlock(&jvmMutex);
     if (!env) {
-        fprintf(stderr, "getJNIEnv: getGlobalJNIEnv failed\n");
-        return NULL;
+      fprintf(stderr, "getJNIEnv: getGlobalJNIEnv failed\n");
+      return NULL;
     }
-    tls = calloc(1, sizeof(struct hdfsTls));
-    if (!tls) {
-        fprintf(stderr, "getJNIEnv: OOM allocating %zd bytes\n",
-                sizeof(struct hdfsTls));
-        return NULL;
-    }
-    tls->env = env;
-    ret = pthread_setspecific(gTlsKey, tls);
-    if (ret) {
-        fprintf(stderr, "getJNIEnv: pthread_setspecific failed with "
-            "error code %d\n", ret);
-        hdfsThreadDestructor(tls);
-        return NULL;
+    if (threadLocalStorageSet(env)) {
+      return NULL;
     }
-#ifdef HAVE_BETTER_TLS
-    quickTls = tls;
-#endif
+    THREAD_LOCAL_STORAGE_SET_QUICK(env);
     return env;
 }
 

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h

@@ -24,8 +24,6 @@
 
 #include <stdlib.h>
 #include <stdarg.h>
-#include <search.h>
-#include <pthread.h>
 #include <errno.h>
 
 #define PATH_SEPARATOR ':'

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c

@@ -21,6 +21,7 @@
 #include "hdfs_test.h"
 #include "jni_helper.h"
 #include "native_mini_dfs.h"
+#include "platform.h"
 
 #include <errno.h>
 #include <jni.h>
@@ -347,10 +348,11 @@ error_dlr_nn:
 int nmdConfigureHdfsBuilder(struct NativeMiniDfsCluster *cl,
                             struct hdfsBuilder *bld)
 {
-    int port, ret;
+    int ret;
+    tPort port;
 
     hdfsBuilderSetNameNode(bld, "localhost");
-    port = nmdGetNameNodePort(cl);
+    port = (tPort)nmdGetNameNodePort(cl);
     if (port < 0) {
       fprintf(stderr, "nmdGetNameNodePort failed with error %d\n", -port);
       return EIO;

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/mutexes.h

@@ -0,0 +1,55 @@
+/**
+ * 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 LIBHDFS_MUTEXES_H
+#define LIBHDFS_MUTEXES_H
+
+/*
+ * Defines abstraction over platform-specific mutexes.  libhdfs has no formal
+ * initialization function that users would call from a single-threaded context
+ * to initialize the library.  This creates a challenge for bootstrapping the
+ * mutexes.  To address this, all required mutexes are pre-defined here with
+ * external storage.  Platform-specific implementations must guarantee that the
+ * mutexes are initialized via static initialization.
+ */
+
+#include "platform.h"
+
+/** Mutex protecting the class reference hash table. */
+extern mutex hdfsHashMutex;
+
+/** Mutex protecting singleton JVM instance. */
+extern mutex jvmMutex;
+
+/**
+ * Locks a mutex.
+ *
+ * @param m mutex
+ * @return 0 if successful, non-zero otherwise
+ */
+int mutexLock(mutex *m);
+
+/**
+ * Unlocks a mutex.
+ *
+ * @param m mutex
+ * @return 0 if successful, non-zero otherwise
+ */
+int mutexUnlock(mutex *m);
+
+#endif

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/mutexes.c

@@ -0,0 +1,43 @@
+/**
+ * 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 "os/mutexes.h"
+
+#include <pthread.h>
+#include <stdio.h>
+
+mutex hdfsHashMutex = PTHREAD_MUTEX_INITIALIZER;
+mutex jvmMutex = PTHREAD_MUTEX_INITIALIZER;
+
+int mutexLock(mutex *m) {
+  int ret = pthread_mutex_lock(m);
+  if (ret) {
+    fprintf(stderr, "mutexLock: pthread_mutex_lock failed with error %d\n",
+      ret);
+  }
+  return ret;
+}
+
+int mutexUnlock(mutex *m) {
+  int ret = pthread_mutex_unlock(m);
+  if (ret) {
+    fprintf(stderr, "mutexUnlock: pthread_mutex_unlock failed with error %d\n",
+      ret);
+  }
+  return ret;
+}

+ 34 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/platform.h

@@ -0,0 +1,34 @@
+/**
+ * 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 LIBHDFS_PLATFORM_H
+#define LIBHDFS_PLATFORM_H
+
+#include <pthread.h>
+
+/* Use gcc type-checked format arguments. */
+#define TYPE_CHECKED_PRINTF_FORMAT(formatArg, varArgs) \
+  __attribute__((format(printf, formatArg, varArgs)))
+
+/*
+ * Mutex and thread data types defined by pthreads.
+ */
+typedef pthread_mutex_t mutex;
+typedef pthread_t threadId;
+
+#endif

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/thread.c

@@ -0,0 +1,52 @@
+/**
+ * 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 "os/thread.h"
+
+#include <pthread.h>
+#include <stdio.h>
+
+/**
+ * Defines a helper function that adapts function pointer provided by caller to
+ * the type required by pthread_create.
+ *
+ * @param toRun thread to run
+ * @return void* result of running thread (always NULL)
+ */
+static void* runThread(void *toRun) {
+  const thread *t = toRun;
+  t->start(t->arg);
+  return NULL;
+}
+
+int threadCreate(thread *t) {
+  int ret;
+  ret = pthread_create(&t->id, NULL, runThread, t);
+  if (ret) {
+    fprintf(stderr, "threadCreate: pthread_create failed with error %d\n", ret);
+  }
+  return ret;
+}
+
+int threadJoin(const thread *t) {
+  int ret = pthread_join(t->id, NULL);
+  if (ret) {
+    fprintf(stderr, "threadJoin: pthread_join failed with error %d\n", ret);
+  }
+  return ret;
+}

+ 80 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/posix/thread_local_storage.c

@@ -0,0 +1,80 @@
+/**
+ * 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 "os/thread_local_storage.h"
+
+#include <jni.h>
+#include <pthread.h>
+#include <stdio.h>
+
+/** Key that allows us to retrieve thread-local storage */
+static pthread_key_t gTlsKey;
+
+/** nonzero if we succeeded in initializing gTlsKey. Protected by the jvmMutex */
+static int gTlsKeyInitialized = 0;
+
+/**
+ * The function that is called whenever a thread with libhdfs thread local data
+ * is destroyed.
+ *
+ * @param v         The thread-local data
+ */
+static void hdfsThreadDestructor(void *v)
+{
+  JavaVM *vm;
+  JNIEnv *env = v;
+  jint ret;
+
+  ret = (*env)->GetJavaVM(env, &vm);
+  if (ret) {
+    fprintf(stderr, "hdfsThreadDestructor: GetJavaVM failed with error %d\n",
+      ret);
+    (*env)->ExceptionDescribe(env);
+  } else {
+    (*vm)->DetachCurrentThread(vm);
+  }
+}
+
+int threadLocalStorageGet(JNIEnv **env)
+{
+  int ret = 0;
+  if (!gTlsKeyInitialized) {
+    ret = pthread_key_create(&gTlsKey, hdfsThreadDestructor);
+    if (ret) {
+      fprintf(stderr,
+        "threadLocalStorageGet: pthread_key_create failed with error %d\n",
+        ret);
+      return ret;
+    }
+    gTlsKeyInitialized = 1;
+  }
+  *env = pthread_getspecific(gTlsKey);
+  return ret;
+}
+
+int threadLocalStorageSet(JNIEnv *env)
+{
+  int ret = pthread_setspecific(gTlsKey, env);
+  if (ret) {
+    fprintf(stderr,
+      "threadLocalStorageSet: pthread_setspecific failed with error %d\n",
+      ret);
+    hdfsThreadDestructor(env);
+  }
+  return ret;
+}

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/thread.h

@@ -0,0 +1,54 @@
+/**
+ * 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 LIBHDFS_THREAD_H
+#define LIBHDFS_THREAD_H
+
+/*
+ * Defines abstraction over platform-specific threads.
+ */
+
+#include "platform.h"
+
+/** Pointer to function to run in thread. */
+typedef void (*threadProcedure)(void *);
+
+/** Structure containing a thread's ID, starting address and argument. */
+typedef struct {
+  threadId id;
+  threadProcedure start;
+  void *arg;
+} thread;
+
+/**
+ * Creates and immediately starts a new thread.
+ *
+ * @param t thread to create
+ * @return 0 if successful, non-zero otherwise
+ */
+int threadCreate(thread *t);
+
+/**
+ * Joins to the given thread, blocking if necessary.
+ *
+ * @param t thread to join
+ * @return 0 if successful, non-zero otherwise
+ */
+int threadJoin(const thread *t);
+
+#endif

+ 75 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/thread_local_storage.h

@@ -0,0 +1,75 @@
+/**
+ * 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 LIBHDFS_THREAD_LOCAL_STORAGE_H
+#define LIBHDFS_THREAD_LOCAL_STORAGE_H
+
+/*
+ * Defines abstraction over platform-specific thread-local storage.  libhdfs
+ * currently only needs thread-local storage for a single piece of data: the
+ * thread's JNIEnv.  For simplicity, this interface is defined in terms of
+ * JNIEnv, not general-purpose thread-local storage of any arbitrary data.
+ */
+
+#include <jni.h>
+
+/*
+ * Most operating systems support the more efficient __thread construct, which
+ * is initialized by the linker.  The following macros use this technique on the
+ * operating systems that support it.
+ */
+#ifdef HAVE_BETTER_TLS
+  #define THREAD_LOCAL_STORAGE_GET_QUICK() \
+    static __thread JNIEnv *quickTlsEnv = NULL; \
+    { \
+      if (quickTlsEnv) { \
+        return quickTlsEnv; \
+      } \
+    }
+
+  #define THREAD_LOCAL_STORAGE_SET_QUICK(env) \
+    { \
+      quickTlsEnv = (env); \
+    }
+#else
+  #define THREAD_LOCAL_STORAGE_GET_QUICK()
+  #define THREAD_LOCAL_STORAGE_SET_QUICK(env)
+#endif
+
+/**
+ * Gets the JNIEnv in thread-local storage for the current thread.  If the call
+ * succeeds, and there is a JNIEnv associated with this thread, then returns 0
+ * and populates env.  If the call succeeds, but there is no JNIEnv associated
+ * with this thread, then returns 0 and sets JNIEnv to NULL.  If the call fails,
+ * then returns non-zero.  Only one thread at a time may execute this function.
+ * The caller is responsible for enforcing mutual exclusion.
+ *
+ * @param env JNIEnv out parameter
+ * @return 0 if successful, non-zero otherwise
+ */
+int threadLocalStorageGet(JNIEnv **env);
+
+/**
+ * Sets the JNIEnv in thread-local storage for the current thread.
+ *
+ * @param env JNIEnv to set
+ * @return 0 if successful, non-zero otherwise
+ */
+int threadLocalStorageSet(JNIEnv *env);
+
+#endif

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/inttypes.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.
+ */
+
+#ifndef LIBHDFS_INTTYPES_H
+#define LIBHDFS_INTTYPES_H
+
+/* On Windows, inttypes.h does not exist, so manually define what we need. */
+
+#define PRId64 "I64d"
+#define PRIu64 "I64u"
+typedef unsigned __int64 uint64_t;
+
+#endif

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/mutexes.c

@@ -0,0 +1,52 @@
+/**
+ * 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 "os/mutexes.h"
+
+#include <windows.h>
+
+mutex hdfsHashMutex;
+mutex jvmMutex;
+
+/**
+ * Unfortunately, there is no simple static initializer for a critical section.
+ * Instead, the API requires calling InitializeCriticalSection.  Since libhdfs
+ * lacks an explicit initialization function, there is no obvious existing place
+ * for the InitializeCriticalSection calls.  To work around this, we define an
+ * initialization function and instruct the linker to set a pointer to that
+ * function as a user-defined global initializer.  See discussion of CRT
+ * Initialization:
+ * http://msdn.microsoft.com/en-us/library/bb918180.aspx
+ */
+static void __cdecl initializeMutexes(void) {
+  InitializeCriticalSection(&hdfsHashMutex);
+  InitializeCriticalSection(&jvmMutex);
+}
+#pragma section(".CRT$XCU", read)
+__declspec(allocate(".CRT$XCU"))
+const void (__cdecl *pInitialize)(void) = initializeMutexes;
+
+int mutexLock(mutex *m) {
+  EnterCriticalSection(m);
+  return 0;
+}
+
+int mutexUnlock(mutex *m) {
+  LeaveCriticalSection(m);
+  return 0;
+}

+ 86 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/platform.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 LIBHDFS_PLATFORM_H
+#define LIBHDFS_PLATFORM_H
+
+#include <stdio.h>
+#include <windows.h>
+#include <winsock.h>
+
+/*
+ * O_ACCMODE defined to match Linux definition.
+ */
+#ifndef O_ACCMODE
+#define O_ACCMODE 0x0003
+#endif
+
+/*
+ * Windows has a different name for its maximum path length constant.
+ */
+#ifndef PATH_MAX
+#define PATH_MAX MAX_PATH
+#endif
+
+/*
+ * Windows does not define EDQUOT and ESTALE in errno.h.  The closest equivalents
+ * are these constants from winsock.h.
+ */
+#ifndef EDQUOT
+#define EDQUOT WSAEDQUOT
+#endif
+
+#ifndef ESTALE
+#define ESTALE WSAESTALE
+#endif
+
+/*
+ * gcc-style type-checked format arguments are not supported on Windows, so just
+ * stub this macro.
+ */
+#define TYPE_CHECKED_PRINTF_FORMAT(formatArg, varArgs)
+
+/*
+ * Define macros for various string formatting functions not defined on Windows.
+ * Where possible, we reroute to one of the secure CRT variants.  On Windows,
+ * the preprocessor does support variadic macros, even though they weren't
+ * defined until C99.
+ */
+#define snprintf(str, size, format, ...) \
+  _snprintf_s((str), (size), _TRUNCATE, (format), __VA_ARGS__)
+#define strncpy(dest, src, n) \
+  strncpy_s((dest), (n), (src), _TRUNCATE)
+#define strtok_r(str, delim, saveptr) \
+  strtok_s((str), (delim), (saveptr))
+#define vsnprintf(str, size, format, ...) \
+  vsnprintf_s((str), (size), _TRUNCATE, (format), __VA_ARGS__)
+
+/*
+ * Mutex data type defined as Windows CRITICAL_SECTION.   A critical section (not
+ * Windows mutex) is used, because libhdfs only needs synchronization of multiple
+ * threads within a single process, not synchronization across process
+ * boundaries.
+ */
+typedef CRITICAL_SECTION mutex;
+
+/*
+ * Thread data type defined as HANDLE to a Windows thread.
+ */
+typedef HANDLE threadId;
+
+#endif

+ 66 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c

@@ -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.
+ */
+
+#include "os/thread.h"
+
+#include <stdio.h>
+#include <windows.h>
+
+/**
+ * Defines a helper function that adapts function pointer provided by caller to
+ * the type required by CreateThread.
+ *
+ * @param toRun thread to run
+ * @return DWORD result of running thread (always 0)
+ */
+static DWORD runThread(LPVOID toRun) {
+  const thread *t = toRun;
+  t->start(t->arg);
+  return 0;
+}
+
+int threadCreate(thread *t) {
+  DWORD ret = 0;
+  HANDLE h;
+  h = CreateThread(NULL, 0, runThread, t, 0, NULL);
+  if (h) {
+    t->id = h;
+  } else {
+    ret = GetLastError();
+    fprintf(stderr, "threadCreate: CreateThread failed with error %d\n", ret);
+  }
+  return ret;
+}
+
+int threadJoin(const thread *t) {
+  DWORD ret = WaitForSingleObject(t->id, INFINITE);
+  switch (ret) {
+  case WAIT_OBJECT_0:
+    break;
+  case WAIT_FAILED:
+    ret = GetLastError();
+    fprintf(stderr, "threadJoin: WaitForSingleObject failed with error %d\n",
+      ret);
+    break;
+  default:
+    fprintf(stderr, "threadJoin: WaitForSingleObject unexpected error %d\n",
+      ret);
+    break;
+  }
+  return ret;
+}

+ 164 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread_local_storage.c

@@ -0,0 +1,164 @@
+/**
+ * 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 "os/thread_local_storage.h"
+
+#include <jni.h>
+#include <stdio.h>
+#include <windows.h>
+
+/** Key that allows us to retrieve thread-local storage */
+static DWORD gTlsIndex = TLS_OUT_OF_INDEXES;
+
+/**
+ * If the current thread has a JNIEnv in thread-local storage, then detaches the
+ * current thread from the JVM.
+ */
+static void detachCurrentThreadFromJvm()
+{
+  JNIEnv *env = NULL;
+  JavaVM *vm;
+  jint ret;
+  if (threadLocalStorageGet(&env) || !env) {
+    return;
+  }
+  ret = (*env)->GetJavaVM(env, &vm);
+  if (ret) {
+    fprintf(stderr,
+      "detachCurrentThreadFromJvm: GetJavaVM failed with error %d\n",
+      ret);
+    (*env)->ExceptionDescribe(env);
+  } else {
+    (*vm)->DetachCurrentThread(vm);
+  }
+}
+
+/**
+ * Unlike pthreads, the Windows API does not seem to provide a convenient way to
+ * hook a callback onto thread shutdown.  However, the Windows portable
+ * executable format does define a concept of thread-local storage callbacks.
+ * Here, we define a function and instruct the linker to set a pointer to that
+ * function in the segment for thread-local storage callbacks.  See page 85 of
+ * Microsoft Portable Executable and Common Object File Format Specification:
+ * http://msdn.microsoft.com/en-us/gg463119.aspx
+ * This technique only works for implicit linking (OS loads DLL on demand), not
+ * for explicit linking (user code calls LoadLibrary directly).  This effectively
+ * means that we have a known limitation: libhdfs may not work correctly if a
+ * Windows application attempts to use it via explicit linking.
+ *
+ * @param h module handle
+ * @param reason the reason for calling the callback
+ * @param pv reserved, unused
+ */
+static void NTAPI tlsCallback(PVOID h, DWORD reason, PVOID pv)
+{
+  DWORD tlsIndex;
+  switch (reason) {
+  case DLL_THREAD_DETACH:
+    detachCurrentThreadFromJvm();
+    break;
+  case DLL_PROCESS_DETACH:
+    detachCurrentThreadFromJvm();
+    tlsIndex = gTlsIndex;
+    gTlsIndex = TLS_OUT_OF_INDEXES;
+    if (!TlsFree(tlsIndex)) {
+      fprintf(stderr, "tlsCallback: TlsFree failed with error %d\n",
+        GetLastError());
+    }
+    break;
+  default:
+    break;
+  }
+}
+
+/*
+ * A variable named _tls_used contains the TLS directory, which contains a list
+ * of pointers to callback functions.  Normally, the linker won't retain this
+ * variable unless the executable has implicit thread-local variables, defined
+ * using the __declspec(thread) extended storage-class modifier.  libhdfs
+ * doesn't use __declspec(thread), and we have no guarantee that the executable
+ * linked to libhdfs will use __declspec(thread).  By forcing the linker to
+ * reference _tls_used, we guarantee that the binary retains the TLS directory.
+ * See Microsoft Visual Studio 10.0/VC/crt/src/tlssup.c .
+ */
+#pragma comment(linker, "/INCLUDE:_tls_used")
+
+/*
+ * We must retain a pointer to the callback function.  Force the linker to keep
+ * this symbol, even though it appears that nothing in our source code uses it.
+ */
+#pragma comment(linker, "/INCLUDE:pTlsCallback")
+
+/*
+ * Define constant pointer to our callback, and tell the linker to pin it into
+ * the TLS directory so that it receives thread callbacks.  Use external linkage
+ * to protect against the linker discarding the seemingly unused symbol.
+ */
+#pragma const_seg(".CRT$XLB")
+extern const PIMAGE_TLS_CALLBACK pTlsCallback;
+const PIMAGE_TLS_CALLBACK pTlsCallback = tlsCallback;
+#pragma const_seg()
+
+int threadLocalStorageGet(JNIEnv **env)
+{
+  LPVOID tls;
+  DWORD ret;
+  if (TLS_OUT_OF_INDEXES == gTlsIndex) {
+    gTlsIndex = TlsAlloc();
+    if (TLS_OUT_OF_INDEXES == gTlsIndex) {
+      fprintf(stderr,
+        "threadLocalStorageGet: TlsAlloc failed with error %d\n",
+        TLS_OUT_OF_INDEXES);
+      return TLS_OUT_OF_INDEXES;
+    }
+  }
+  tls = TlsGetValue(gTlsIndex);
+  if (tls) {
+    *env = tls;
+    return 0;
+  } else {
+    ret = GetLastError();
+    if (ERROR_SUCCESS == ret) {
+      /* Thread-local storage contains NULL, because we haven't set it yet. */
+      *env = NULL;
+      return 0;
+    } else {
+      /*
+       * The API call failed.  According to documentation, TlsGetValue cannot
+       * fail as long as the index is a valid index from a successful TlsAlloc
+       * call.  This error handling is purely defensive.
+       */
+      fprintf(stderr,
+        "threadLocalStorageGet: TlsGetValue failed with error %d\n", ret);
+      return ret;
+    }
+  }
+}
+
+int threadLocalStorageSet(JNIEnv *env)
+{
+  DWORD ret = 0;
+  if (!TlsSetValue(gTlsIndex, (LPVOID)env)) {
+    ret = GetLastError();
+    fprintf(stderr,
+      "threadLocalStorageSet: TlsSetValue failed with error %d\n",
+      ret);
+    detachCurrentThreadFromJvm(env);
+  }
+  return ret;
+}

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/unistd.h

@@ -0,0 +1,29 @@
+/**
+ * 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 LIBHDFS_UNISTD_H
+#define LIBHDFS_UNISTD_H
+
+/* On Windows, unistd.h does not exist, so manually define what we need. */
+
+#include <process.h> /* Declares getpid(). */
+#include <windows.h>
+
+/* Re-route sleep to Sleep, converting units from seconds to milliseconds. */
+#define sleep(seconds) Sleep((seconds) * 1000)
+#endif

+ 119 - 113
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_ops.c

@@ -18,6 +18,7 @@
 
 #include "hdfs.h" 
 #include "hdfs_test.h" 
+#include "platform.h"
 
 #include <inttypes.h>
 #include <jni.h>
@@ -28,12 +29,13 @@
 #include <unistd.h>
 
 void permission_disp(short permissions, char *rtr) {
-  rtr[9] = '\0';
   int i;
+  short permissionsId;
+  char* perm;
+  rtr[9] = '\0';
   for(i=2;i>=0;i--)
     {
-      short permissionsId = permissions >> (i * 3) & (short)7;
-      char* perm;
+      permissionsId = permissions >> (i * 3) & (short)7;
       switch(permissionsId) {
       case 7:
         perm = "rwx"; break;
@@ -60,35 +62,56 @@ void permission_disp(short permissions, char *rtr) {
 } 
 
 int main(int argc, char **argv) {
-    char buffer[32];
-    tSize num_written_bytes;
-
-    hdfsFS fs = hdfsConnectNewInstance("default", 0);
+    const char *writePath = "/tmp/testfile.txt";
+    const char *fileContents = "Hello, World!";
+    const char *readPath = "/tmp/testfile.txt";
+    const char *srcPath = "/tmp/testfile.txt";
+    const char *dstPath = "/tmp/testfile2.txt";
+    const char *slashTmp = "/tmp";
+    const char *newDirectory = "/tmp/newdir";
+    const char *newOwner = "root";
+    const char *tuser = "nobody";
+    const char *appendPath = "/tmp/appends";
+    const char *userPath = "/tmp/usertestfile.txt";
+
+    char buffer[32], buffer2[256], rdbuffer[32];
+    tSize num_written_bytes, num_read_bytes;
+    hdfsFS fs, lfs;
+    hdfsFile writeFile, readFile, localFile, appendFile, userFile;
+    tOffset currentPos, seekPos;
+    int exists, totalResult, result, numEntries, i, j;
+    const char *resp;
+    hdfsFileInfo *fileInfo, *fileList, *finfo;
+    char *buffer3;
+    char permissions[10];
+    char ***hosts;
+    short newPerm = 0666;
+    tTime newMtime, newAtime;
+
+    fs = hdfsConnectNewInstance("default", 0);
     if(!fs) {
         fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
         exit(-1);
     } 
  
-    hdfsFS lfs = hdfsConnectNewInstance(NULL, 0);
+    lfs = hdfsConnectNewInstance(NULL, 0);
     if(!lfs) {
         fprintf(stderr, "Oops! Failed to connect to 'local' hdfs!\n");
         exit(-1);
     } 
 
-    const char* writePath = "/tmp/testfile.txt";
-    const char* fileContents = "Hello, World!";
-
     {
         //Write tests
         
-        hdfsFile writeFile = hdfsOpenFile(fs, writePath, O_WRONLY|O_CREAT, 0, 0, 0);
+        writeFile = hdfsOpenFile(fs, writePath, O_WRONLY|O_CREAT, 0, 0, 0);
         if(!writeFile) {
             fprintf(stderr, "Failed to open %s for writing!\n", writePath);
             exit(-1);
         }
         fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
         num_written_bytes =
-          hdfsWrite(fs, writeFile, (void*)fileContents, strlen(fileContents)+1);
+          hdfsWrite(fs, writeFile, (void*)fileContents,
+            (tSize)(strlen(fileContents)+1));
         if (num_written_bytes != strlen(fileContents) + 1) {
           fprintf(stderr, "Failed to write correct number of bytes - expected %d, got %d\n",
                   (int)(strlen(fileContents) + 1), (int)num_written_bytes);
@@ -96,7 +119,7 @@ int main(int argc, char **argv) {
         }
         fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
 
-        tOffset currentPos = -1;
+        currentPos = -1;
         if ((currentPos = hdfsTell(fs, writeFile)) == -1) {
             fprintf(stderr, 
                     "Failed to get current file position correctly! Got %ld!\n",
@@ -123,15 +146,14 @@ int main(int argc, char **argv) {
     {
         //Read tests
         
-        const char* readPath = "/tmp/testfile.txt";
-        int exists = hdfsExists(fs, readPath);
+        exists = hdfsExists(fs, readPath);
 
         if (exists) {
           fprintf(stderr, "Failed to validate existence of %s\n", readPath);
           exit(-1);
         }
 
-        hdfsFile readFile = hdfsOpenFile(fs, readPath, O_RDONLY, 0, 0, 0);
+        readFile = hdfsOpenFile(fs, readPath, O_RDONLY, 0, 0, 0);
         if (!readFile) {
             fprintf(stderr, "Failed to open %s for reading!\n", readPath);
             exit(-1);
@@ -146,13 +168,13 @@ int main(int argc, char **argv) {
 
         fprintf(stderr, "hdfsAvailable: %d\n", hdfsAvailable(fs, readFile));
 
-        tOffset seekPos = 1;
+        seekPos = 1;
         if(hdfsSeek(fs, readFile, seekPos)) {
             fprintf(stderr, "Failed to seek %s for reading!\n", readPath);
             exit(-1);
         }
 
-        tOffset currentPos = -1;
+        currentPos = -1;
         if((currentPos = hdfsTell(fs, readFile)) != seekPos) {
             fprintf(stderr, 
                     "Failed to get current file position correctly! Got %ld!\n", 
@@ -175,7 +197,7 @@ int main(int argc, char **argv) {
             exit(-1);
         }
         memset(buffer, 0, sizeof(buffer));
-        tSize num_read_bytes = hdfsRead(fs, readFile, (void*)buffer,
+        num_read_bytes = hdfsRead(fs, readFile, (void*)buffer,
                 sizeof(buffer));
         if (strncmp(fileContents, buffer, strlen(fileContents)) != 0) {
             fprintf(stderr, "Failed to read (direct). Expected %s but got %s (%d bytes)\n",
@@ -208,14 +230,14 @@ int main(int argc, char **argv) {
         hdfsCloseFile(fs, readFile);
 
         // Test correct behaviour for unsupported filesystems
-        hdfsFile localFile = hdfsOpenFile(lfs, writePath, O_WRONLY|O_CREAT, 0, 0, 0);
+        localFile = hdfsOpenFile(lfs, writePath, O_WRONLY|O_CREAT, 0, 0, 0);
         if(!localFile) {
             fprintf(stderr, "Failed to open %s for writing!\n", writePath);
             exit(-1);
         }
 
         num_written_bytes = hdfsWrite(lfs, localFile, (void*)fileContents,
-                                      strlen(fileContents) + 1);
+                                      (tSize)(strlen(fileContents) + 1));
 
         hdfsCloseFile(lfs, localFile);
         localFile = hdfsOpenFile(lfs, writePath, O_RDONLY, 0, 0, 0);
@@ -229,50 +251,43 @@ int main(int argc, char **argv) {
         hdfsCloseFile(lfs, localFile);
     }
 
-    int totalResult = 0;
-    int result = 0;
+    totalResult = 0;
+    result = 0;
     {
         //Generic file-system operations
 
-        const char* srcPath = "/tmp/testfile.txt";
-        const char* dstPath = "/tmp/testfile2.txt";
-
-        fprintf(stderr, "hdfsCopy(remote-local): %s\n", ((result = hdfsCopy(fs, srcPath, lfs, srcPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsCopy(remote-local): %s\n", ((result = hdfsCopy(fs, srcPath, lfs, srcPath)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsCopy(remote-remote): %s\n", ((result = hdfsCopy(fs, srcPath, fs, dstPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsCopy(remote-remote): %s\n", ((result = hdfsCopy(fs, srcPath, fs, dstPath)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsMove(local-local): %s\n", ((result = hdfsMove(lfs, srcPath, lfs, dstPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsMove(local-local): %s\n", ((result = hdfsMove(lfs, srcPath, lfs, dstPath)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsMove(remote-local): %s\n", ((result = hdfsMove(fs, srcPath, lfs, srcPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsMove(remote-local): %s\n", ((result = hdfsMove(fs, srcPath, lfs, srcPath)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
-        fprintf(stderr, "hdfsRename: %s\n", ((result = hdfsRename(fs, dstPath, srcPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsRename: %s\n", ((result = hdfsRename(fs, dstPath, srcPath)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsCopy(remote-remote): %s\n", ((result = hdfsCopy(fs, srcPath, fs, dstPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsCopy(remote-remote): %s\n", ((result = hdfsCopy(fs, srcPath, fs, dstPath)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
-        const char* slashTmp = "/tmp";
-        const char* newDirectory = "/tmp/newdir";
-        fprintf(stderr, "hdfsCreateDirectory: %s\n", ((result = hdfsCreateDirectory(fs, newDirectory)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsCreateDirectory: %s\n", ((result = hdfsCreateDirectory(fs, newDirectory)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
-        fprintf(stderr, "hdfsSetReplication: %s\n", ((result = hdfsSetReplication(fs, srcPath, 2)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsSetReplication: %s\n", ((result = hdfsSetReplication(fs, srcPath, 2)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
-        char buffer[256];
-        const char *resp;
-        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", ((resp = hdfsGetWorkingDirectory(fs, buffer, sizeof(buffer))) ? buffer : "Failed!"));
+        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", ((resp = hdfsGetWorkingDirectory(fs, buffer2, sizeof(buffer2))) != 0 ? buffer2 : "Failed!"));
         totalResult += (resp ? 0 : 1);
-        fprintf(stderr, "hdfsSetWorkingDirectory: %s\n", ((result = hdfsSetWorkingDirectory(fs, slashTmp)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsSetWorkingDirectory: %s\n", ((result = hdfsSetWorkingDirectory(fs, slashTmp)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", ((resp = hdfsGetWorkingDirectory(fs, buffer, sizeof(buffer))) ? buffer : "Failed!"));
+        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", ((resp = hdfsGetWorkingDirectory(fs, buffer2, sizeof(buffer2))) != 0 ? buffer2 : "Failed!"));
         totalResult += (resp ? 0 : 1);
 
         fprintf(stderr, "hdfsGetDefaultBlockSize: %ld\n", hdfsGetDefaultBlockSize(fs));
         fprintf(stderr, "hdfsGetCapacity: %ld\n", hdfsGetCapacity(fs));
         fprintf(stderr, "hdfsGetUsed: %ld\n", hdfsGetUsed(fs));
 
-        hdfsFileInfo *fileInfo = NULL;
+        fileInfo = NULL;
         if((fileInfo = hdfsGetPathInfo(fs, slashTmp)) != NULL) {
             fprintf(stderr, "hdfsGetPathInfo - SUCCESS!\n");
             fprintf(stderr, "Name: %s, ", fileInfo->mName);
@@ -283,7 +298,6 @@ int main(int argc, char **argv) {
             fprintf(stderr, "LastMod: %s", ctime(&fileInfo->mLastMod)); 
             fprintf(stderr, "Owner: %s, ", fileInfo->mOwner);
             fprintf(stderr, "Group: %s, ", fileInfo->mGroup);
-            char permissions[10];
             permission_disp(fileInfo->mPermissions, permissions);
             fprintf(stderr, "Permissions: %d (%s)\n", fileInfo->mPermissions, permissions);
             hdfsFreeFileInfo(fileInfo, 1);
@@ -292,10 +306,8 @@ int main(int argc, char **argv) {
             fprintf(stderr, "waah! hdfsGetPathInfo for %s - FAILED!\n", slashTmp);
         }
 
-        hdfsFileInfo *fileList = 0;
-        int numEntries = 0;
+        fileList = 0;
         if((fileList = hdfsListDirectory(fs, slashTmp, &numEntries)) != NULL) {
-            int i = 0;
             for(i=0; i < numEntries; ++i) {
                 fprintf(stderr, "Name: %s, ", fileList[i].mName);
                 fprintf(stderr, "Type: %c, ", (char)fileList[i].mKind);
@@ -305,7 +317,6 @@ int main(int argc, char **argv) {
                 fprintf(stderr, "LastMod: %s", ctime(&fileList[i].mLastMod));
                 fprintf(stderr, "Owner: %s, ", fileList[i].mOwner);
                 fprintf(stderr, "Group: %s, ", fileList[i].mGroup);
-                char permissions[10];
                 permission_disp(fileList[i].mPermissions, permissions);
                 fprintf(stderr, "Permissions: %d (%s)\n", fileList[i].mPermissions, permissions);
             }
@@ -319,12 +330,12 @@ int main(int argc, char **argv) {
             }
         }
 
-        char*** hosts = hdfsGetHosts(fs, srcPath, 0, 1);
+        hosts = hdfsGetHosts(fs, srcPath, 0, 1);
         if(hosts) {
             fprintf(stderr, "hdfsGetHosts - SUCCESS! ... \n");
-            int i=0; 
+            i=0; 
             while(hosts[i]) {
-                int j = 0;
+                j = 0;
                 while(hosts[i][j]) {
                     fprintf(stderr, 
                             "\thosts[%d][%d] - %s\n", i, j, hosts[i][j]);
@@ -337,131 +348,129 @@ int main(int argc, char **argv) {
             fprintf(stderr, "waah! hdfsGetHosts - FAILED!\n");
         }
        
-        char *newOwner = "root";
         // setting tmp dir to 777 so later when connectAsUser nobody, we can write to it
-        short newPerm = 0666;
 
         // chown write
-        fprintf(stderr, "hdfsChown: %s\n", ((result = hdfsChown(fs, writePath, NULL, "users")) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsChown: %s\n", ((result = hdfsChown(fs, writePath, NULL, "users")) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsChown: %s\n", ((result = hdfsChown(fs, writePath, newOwner, NULL)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsChown: %s\n", ((result = hdfsChown(fs, writePath, newOwner, NULL)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
         // chmod write
-        fprintf(stderr, "hdfsChmod: %s\n", ((result = hdfsChmod(fs, writePath, newPerm)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsChmod: %s\n", ((result = hdfsChmod(fs, writePath, newPerm)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
 
 
         sleep(2);
-        tTime newMtime = time(NULL);
-        tTime newAtime = time(NULL);
+        newMtime = time(NULL);
+        newAtime = time(NULL);
 
         // utime write
-        fprintf(stderr, "hdfsUtime: %s\n", ((result = hdfsUtime(fs, writePath, newMtime, newAtime)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsUtime: %s\n", ((result = hdfsUtime(fs, writePath, newMtime, newAtime)) != 0 ? "Failed!" : "Success!"));
 
         totalResult += result;
 
         // chown/chmod/utime read
-        hdfsFileInfo *finfo = hdfsGetPathInfo(fs, writePath);
+        finfo = hdfsGetPathInfo(fs, writePath);
 
-        fprintf(stderr, "hdfsChown read: %s\n", ((result = (strcmp(finfo->mOwner, newOwner) != 0)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsChown read: %s\n", ((result = (strcmp(finfo->mOwner, newOwner))) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
-        fprintf(stderr, "hdfsChmod read: %s\n", ((result = (finfo->mPermissions != newPerm)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsChmod read: %s\n", ((result = (finfo->mPermissions != newPerm)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
         // will later use /tmp/ as a different user so enable it
-        fprintf(stderr, "hdfsChmod: %s\n", ((result = hdfsChmod(fs, "/tmp/", 0777)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsChmod: %s\n", ((result = hdfsChmod(fs, "/tmp/", 0777)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
         fprintf(stderr,"newMTime=%ld\n",newMtime);
         fprintf(stderr,"curMTime=%ld\n",finfo->mLastMod);
 
 
-        fprintf(stderr, "hdfsUtime read (mtime): %s\n", ((result = (finfo->mLastMod != newMtime)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsUtime read (mtime): %s\n", ((result = (finfo->mLastMod != newMtime)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
 
         // No easy way to turn on access times from hdfs_test right now
-        //        fprintf(stderr, "hdfsUtime read (atime): %s\n", ((result = (finfo->mLastAccess != newAtime)) ? "Failed!" : "Success!"));
+        //        fprintf(stderr, "hdfsUtime read (atime): %s\n", ((result = (finfo->mLastAccess != newAtime)) != 0 ? "Failed!" : "Success!"));
         //        totalResult += result;
 
         hdfsFreeFileInfo(finfo, 1);
 
         // Clean up
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, newDirectory, 1)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, newDirectory, 1)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, srcPath, 1)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, srcPath, 1)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, srcPath, 1)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, srcPath, 1)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, dstPath, 1)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, dstPath, 1)) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
-        fprintf(stderr, "hdfsExists: %s\n", ((result = hdfsExists(fs, newDirectory)) ? "Success!" : "Failed!"));
+        fprintf(stderr, "hdfsExists: %s\n", ((result = hdfsExists(fs, newDirectory)) != 0 ? "Success!" : "Failed!"));
         totalResult += (result ? 0 : 1);
     }
 
     {
       // TEST APPENDS
-      const char *writePath = "/tmp/appends";
 
       // CREATE
-      hdfsFile writeFile = hdfsOpenFile(fs, writePath, O_WRONLY, 0, 0, 0);
-      if(!writeFile) {
-        fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+      appendFile = hdfsOpenFile(fs, appendPath, O_WRONLY, 0, 0, 0);
+      if(!appendFile) {
+        fprintf(stderr, "Failed to open %s for writing!\n", appendPath);
         exit(-1);
       }
-      fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
+      fprintf(stderr, "Opened %s for writing successfully...\n", appendPath);
 
-      char* buffer = "Hello,";
-      tSize num_written_bytes = hdfsWrite(fs, writeFile, (void*)buffer, strlen(buffer));
+      buffer3 = "Hello,";
+      num_written_bytes = hdfsWrite(fs, appendFile, (void*)buffer3,
+        (tSize)strlen(buffer3));
       fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
 
-      if (hdfsFlush(fs, writeFile)) {
-        fprintf(stderr, "Failed to 'flush' %s\n", writePath); 
+      if (hdfsFlush(fs, appendFile)) {
+        fprintf(stderr, "Failed to 'flush' %s\n", appendPath); 
         exit(-1);
         }
-      fprintf(stderr, "Flushed %s successfully!\n", writePath); 
+      fprintf(stderr, "Flushed %s successfully!\n", appendPath); 
 
-      hdfsCloseFile(fs, writeFile);
+      hdfsCloseFile(fs, appendFile);
 
       // RE-OPEN
-      writeFile = hdfsOpenFile(fs, writePath, O_WRONLY|O_APPEND, 0, 0, 0);
-      if(!writeFile) {
-        fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+      appendFile = hdfsOpenFile(fs, appendPath, O_WRONLY|O_APPEND, 0, 0, 0);
+      if(!appendFile) {
+        fprintf(stderr, "Failed to open %s for writing!\n", appendPath);
         exit(-1);
       }
-      fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
+      fprintf(stderr, "Opened %s for writing successfully...\n", appendPath);
 
-      buffer = " World";
-      num_written_bytes = hdfsWrite(fs, writeFile, (void*)buffer, strlen(buffer) + 1);
+      buffer3 = " World";
+      num_written_bytes = hdfsWrite(fs, appendFile, (void*)buffer3,
+        (tSize)(strlen(buffer3) + 1));
       fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
 
-      if (hdfsFlush(fs, writeFile)) {
-        fprintf(stderr, "Failed to 'flush' %s\n", writePath); 
+      if (hdfsFlush(fs, appendFile)) {
+        fprintf(stderr, "Failed to 'flush' %s\n", appendPath); 
         exit(-1);
       }
-      fprintf(stderr, "Flushed %s successfully!\n", writePath); 
+      fprintf(stderr, "Flushed %s successfully!\n", appendPath); 
 
-      hdfsCloseFile(fs, writeFile);
+      hdfsCloseFile(fs, appendFile);
 
       // CHECK size
-      hdfsFileInfo *finfo = hdfsGetPathInfo(fs, writePath);
-      fprintf(stderr, "fileinfo->mSize: == total %s\n", ((result = (finfo->mSize == strlen("Hello, World") + 1)) ? "Success!" : "Failed!"));
+      finfo = hdfsGetPathInfo(fs, appendPath);
+      fprintf(stderr, "fileinfo->mSize: == total %s\n", ((result = (finfo->mSize == (tOffset)(strlen("Hello, World") + 1))) == 1 ? "Success!" : "Failed!"));
       totalResult += (result ? 0 : 1);
 
       // READ and check data
-      hdfsFile readFile = hdfsOpenFile(fs, writePath, O_RDONLY, 0, 0, 0);
+      readFile = hdfsOpenFile(fs, appendPath, O_RDONLY, 0, 0, 0);
       if (!readFile) {
-        fprintf(stderr, "Failed to open %s for reading!\n", writePath);
+        fprintf(stderr, "Failed to open %s for reading!\n", appendPath);
         exit(-1);
       }
 
-      char rdbuffer[32];
-      tSize num_read_bytes = hdfsRead(fs, readFile, (void*)rdbuffer, sizeof(rdbuffer));
+      num_read_bytes = hdfsRead(fs, readFile, (void*)rdbuffer, sizeof(rdbuffer));
       fprintf(stderr, "Read following %d bytes:\n%s\n", 
               num_read_bytes, rdbuffer);
 
-      fprintf(stderr, "read == Hello, World %s\n", (result = (strcmp(rdbuffer, "Hello, World") == 0)) ? "Success!" : "Failed!");
+      fprintf(stderr, "read == Hello, World %s\n", ((result = (strcmp(rdbuffer, "Hello, World"))) == 0 ? "Success!" : "Failed!"));
 
       hdfsCloseFile(fs, readFile);
 
@@ -478,36 +487,33 @@ int main(int argc, char **argv) {
       // the actual fs user capabilities. Thus just create a file and read
       // the owner is correct.
 
-      const char *tuser = "nobody";
-      const char* writePath = "/tmp/usertestfile.txt";
-
       fs = hdfsConnectAsUserNewInstance("default", 0, tuser);
       if(!fs) {
         fprintf(stderr, "Oops! Failed to connect to hdfs as user %s!\n",tuser);
         exit(-1);
       } 
 
-        hdfsFile writeFile = hdfsOpenFile(fs, writePath, O_WRONLY|O_CREAT, 0, 0, 0);
-        if(!writeFile) {
-            fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+        userFile = hdfsOpenFile(fs, userPath, O_WRONLY|O_CREAT, 0, 0, 0);
+        if(!userFile) {
+            fprintf(stderr, "Failed to open %s for writing!\n", userPath);
             exit(-1);
         }
-        fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
+        fprintf(stderr, "Opened %s for writing successfully...\n", userPath);
 
-        char* buffer = "Hello, World!";
-        tSize num_written_bytes = hdfsWrite(fs, writeFile, (void*)buffer, strlen(buffer)+1);
+        num_written_bytes = hdfsWrite(fs, userFile, (void*)fileContents,
+          (tSize)(strlen(fileContents)+1));
         fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
 
-        if (hdfsFlush(fs, writeFile)) {
-            fprintf(stderr, "Failed to 'flush' %s\n", writePath); 
+        if (hdfsFlush(fs, userFile)) {
+            fprintf(stderr, "Failed to 'flush' %s\n", userPath); 
             exit(-1);
         }
-        fprintf(stderr, "Flushed %s successfully!\n", writePath); 
+        fprintf(stderr, "Flushed %s successfully!\n", userPath); 
 
-        hdfsCloseFile(fs, writeFile);
+        hdfsCloseFile(fs, userFile);
 
-        hdfsFileInfo *finfo = hdfsGetPathInfo(fs, writePath);
-        fprintf(stderr, "hdfs new file user is correct: %s\n", ((result = (strcmp(finfo->mOwner, tuser) != 0)) ? "Failed!" : "Success!"));
+        finfo = hdfsGetPathInfo(fs, userPath);
+        fprintf(stderr, "hdfs new file user is correct: %s\n", ((result = (strcmp(finfo->mOwner, tuser))) != 0 ? "Failed!" : "Success!"));
         totalResult += result;
     }
     

+ 11 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_read.c

@@ -22,35 +22,38 @@
 #include <stdlib.h>
 
 int main(int argc, char **argv) {
+    hdfsFS fs;
+    const char *rfile = argv[1];
+    tSize bufferSize = strtoul(argv[3], NULL, 10);
+    hdfsFile readFile;
+    char* buffer;
+    tSize curSize;
 
     if (argc != 4) {
         fprintf(stderr, "Usage: hdfs_read <filename> <filesize> <buffersize>\n");
         exit(-1);
     }
     
-    hdfsFS fs = hdfsConnect("default", 0);
+    fs = hdfsConnect("default", 0);
     if (!fs) {
         fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
         exit(-1);
     } 
- 
-    const char* rfile = argv[1];
-    tSize bufferSize = strtoul(argv[3], NULL, 10);
-   
-    hdfsFile readFile = hdfsOpenFile(fs, rfile, O_RDONLY, bufferSize, 0, 0);
+
+    readFile = hdfsOpenFile(fs, rfile, O_RDONLY, bufferSize, 0, 0);
     if (!readFile) {
         fprintf(stderr, "Failed to open %s for writing!\n", rfile);
         exit(-2);
     }
 
     // data to be written to the file
-    char* buffer = malloc(sizeof(char) * bufferSize);
+    buffer = malloc(sizeof(char) * bufferSize);
     if(buffer == NULL) {
         return -2;
     }
     
     // read from the file
-    tSize curSize = bufferSize;
+    curSize = bufferSize;
     for (; curSize == bufferSize;) {
         curSize = hdfsRead(fs, readFile, (void*)buffer, curSize);
     }

+ 17 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_write.c

@@ -21,23 +21,31 @@
 #include <limits.h>
 #include <stdio.h>
 #include <stdlib.h>
+#include <sys/types.h>
 
 int main(int argc, char **argv) {
+    hdfsFS fs;
+    const char *writeFileName = argv[1];
+    off_t fileTotalSize = strtoul(argv[2], NULL, 10);
+    long long tmpBufferSize = strtoul(argv[3], NULL, 10);
+    tSize bufferSize;
+    hdfsFile writeFile;
+    char* buffer;
+    int i;
+    off_t nrRemaining;
+    tSize curSize;
+    tSize written;
 
     if (argc != 4) {
         fprintf(stderr, "Usage: hdfs_write <filename> <filesize> <buffersize>\n");
         exit(-1);
     }
     
-    hdfsFS fs = hdfsConnect("default", 0);
+    fs = hdfsConnect("default", 0);
     if (!fs) {
         fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
         exit(-1);
     } 
- 
-    const char* writeFileName = argv[1];
-    off_t fileTotalSize = strtoul(argv[2], NULL, 10);
-    long long tmpBufferSize = strtoul(argv[3], NULL, 10);
 
     // sanity check
     if(fileTotalSize == ULONG_MAX && errno == ERANGE) {
@@ -51,30 +59,27 @@ int main(int argc, char **argv) {
       exit(-3);
     }
 
-    tSize bufferSize = tmpBufferSize;
+    bufferSize = (tSize)tmpBufferSize;
 
-    hdfsFile writeFile = hdfsOpenFile(fs, writeFileName, O_WRONLY, bufferSize, 0, 0);
+    writeFile = hdfsOpenFile(fs, writeFileName, O_WRONLY, bufferSize, 0, 0);
     if (!writeFile) {
         fprintf(stderr, "Failed to open %s for writing!\n", writeFileName);
         exit(-2);
     }
 
     // data to be written to the file
-    char* buffer = malloc(sizeof(char) * bufferSize);
+    buffer = malloc(sizeof(char) * bufferSize);
     if(buffer == NULL) {
         fprintf(stderr, "Could not allocate buffer of size %d\n", bufferSize);
         return -2;
     }
-    int i = 0;
     for (i=0; i < bufferSize; ++i) {
         buffer[i] = 'a' + (i%26);
     }
 
     // write to the file
-    off_t nrRemaining;
     for (nrRemaining = fileTotalSize; nrRemaining > 0; nrRemaining -= bufferSize ) {
-      tSize curSize = ( bufferSize < nrRemaining ) ? bufferSize : (tSize)nrRemaining; 
-      tSize written;
+      curSize = ( bufferSize < nrRemaining ) ? bufferSize : (tSize)nrRemaining; 
       if ((written = hdfsWrite(fs, writeFile, (void*)buffer, curSize)) != curSize) {
         fprintf(stderr, "ERROR: hdfsWrite returned an error on write: %d\n", written);
         exit(-3);

+ 11 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_zerocopy.c

@@ -19,12 +19,12 @@
 #include "expect.h"
 #include "hdfs.h"
 #include "native_mini_dfs.h"
+#include "platform.h"
 
 #include <errno.h>
 #include <inttypes.h>
-#include <semaphore.h>
-#include <pthread.h>
 #include <unistd.h>
+#include <stdint.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
@@ -53,7 +53,7 @@ static uint8_t *getZeroCopyBlockData(int blockIdx)
         exit(1);
     }
     for (i = 0; i < TEST_ZEROCOPY_FULL_BLOCK_SIZE; i++) {
-      buf[i] = blockIdx + (i % 17);
+      buf[i] = (uint8_t)(blockIdx + (i % 17));
     }
     return buf;
 }
@@ -69,18 +69,6 @@ static int getZeroCopyBlockLen(int blockIdx)
     }
 }
 
-static void printBuf(const uint8_t *buf, size_t len) __attribute__((unused));
-
-static void printBuf(const uint8_t *buf, size_t len)
-{
-  size_t i;
-
-  for (i = 0; i < len; i++) {
-    fprintf(stderr, "%02x", buf[i]);
-  }
-  fprintf(stderr, "\n");
-}
-
 static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
 {
     hdfsFile file = NULL;
@@ -127,8 +115,9 @@ static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
     EXPECT_NONNULL(block);
     EXPECT_ZERO(memcmp(block, hadoopRzBufferGet(buffer), SMALL_READ_LEN));
     hadoopRzBufferFree(file, buffer);
-    EXPECT_INT_EQ(TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN,
-                  hdfsTell(fs, file));
+    EXPECT_INT64_EQ(
+          (int64_t)TEST_ZEROCOPY_FULL_BLOCK_SIZE + (int64_t)SMALL_READ_LEN,
+          hdfsTell(fs, file));
     EXPECT_ZERO(expectFileStats(file,
           TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN,
           TEST_ZEROCOPY_FULL_BLOCK_SIZE + SMALL_READ_LEN,
@@ -165,7 +154,7 @@ static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
     free(block);
     block = getZeroCopyBlockData(2);
     EXPECT_NONNULL(block);
-    EXPECT_ZERO(memcmp(block, hadoopRzBufferGet(buffer) +
+    EXPECT_ZERO(memcmp(block, (uint8_t*)hadoopRzBufferGet(buffer) +
         (TEST_ZEROCOPY_FULL_BLOCK_SIZE - SMALL_READ_LEN), SMALL_READ_LEN));
     hadoopRzBufferFree(file, buffer);
 
@@ -219,8 +208,10 @@ int main(void)
 {
     int port;
     struct NativeMiniDfsConf conf = {
-        .doFormat = 1,
-        .configureShortCircuit = 1,
+        1, /* doFormat */
+        0, /* webhdfsEnabled */
+        0, /* namenodeHttpPort */
+        1, /* configureShortCircuit */
     };
     char testFileName[TEST_FILE_NAME_LENGTH];
     hdfsFS fs;

+ 18 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c

@@ -19,11 +19,11 @@
 #include "expect.h"
 #include "hdfs.h"
 #include "native_mini_dfs.h"
+#include "os/thread.h"
 
 #include <errno.h>
 #include <inttypes.h>
-#include <semaphore.h>
-#include <pthread.h>
+#include <stdint.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
@@ -35,8 +35,6 @@
 
 #define TLH_DEFAULT_BLOCK_SIZE 134217728
 
-static sem_t tlhSem;
-
 static struct NativeMiniDfsCluster* tlhCluster;
 
 struct tlhThreadInfo {
@@ -44,18 +42,19 @@ struct tlhThreadInfo {
     int threadIdx;
     /** 0 = thread was successful; error code otherwise */
     int success;
-    /** pthread identifier */
-    pthread_t thread;
+    /** thread identifier */
+    thread theThread;
 };
 
 static int hdfsSingleNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs,
                                      const char *username)
 {
-    int ret, port;
+    int ret;
+    tPort port;
     hdfsFS hdfs;
     struct hdfsBuilder *bld;
     
-    port = nmdGetNameNodePort(cl);
+    port = (tPort)nmdGetNameNodePort(cl);
     if (port < 0) {
         fprintf(stderr, "hdfsSingleNameNodeConnect: nmdGetNameNodePort "
                 "returned error %d\n", port);
@@ -164,7 +163,7 @@ static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
     EXPECT_NONNULL(file);
 
     /* TODO: implement writeFully and use it here */
-    expected = strlen(paths->prefix);
+    expected = (int)strlen(paths->prefix);
     ret = hdfsWrite(fs, file, paths->prefix, expected);
     if (ret < 0) {
         ret = errno;
@@ -186,9 +185,9 @@ static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
 
     EXPECT_ZERO(hdfsFileGetReadStatistics(file, &readStats));
     errno = 0;
-    EXPECT_ZERO(readStats->totalBytesRead);
-    EXPECT_ZERO(readStats->totalLocalBytesRead);
-    EXPECT_ZERO(readStats->totalShortCircuitBytesRead);
+    EXPECT_UINT64_EQ(UINT64_C(0), readStats->totalBytesRead);
+    EXPECT_UINT64_EQ(UINT64_C(0), readStats->totalLocalBytesRead);
+    EXPECT_UINT64_EQ(UINT64_C(0), readStats->totalShortCircuitBytesRead);
     hdfsFileFreeReadStatistics(readStats);
     /* TODO: implement readFully and use it here */
     ret = hdfsRead(fs, file, tmp, sizeof(tmp));
@@ -204,7 +203,7 @@ static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
     }
     EXPECT_ZERO(hdfsFileGetReadStatistics(file, &readStats));
     errno = 0;
-    EXPECT_INT_EQ(expected, readStats->totalBytesRead);
+    EXPECT_UINT64_EQ((uint64_t)expected, readStats->totalBytesRead);
     hdfsFileFreeReadStatistics(readStats);
     EXPECT_ZERO(memcmp(paths->prefix, tmp, expected));
     EXPECT_ZERO(hdfsCloseFile(fs, file));
@@ -262,12 +261,11 @@ static int testHdfsOperationsImpl(struct tlhThreadInfo *ti)
     return 0;
 }
 
-static void *testHdfsOperations(void *v)
+static void testHdfsOperations(void *v)
 {
     struct tlhThreadInfo *ti = (struct tlhThreadInfo*)v;
     int ret = testHdfsOperationsImpl(ti);
     ti->success = ret;
-    return NULL;
 }
 
 static int checkFailures(struct tlhThreadInfo *ti, int tlhNumThreads)
@@ -304,7 +302,7 @@ int main(void)
     const char *tlhNumThreadsStr;
     struct tlhThreadInfo ti[TLH_MAX_THREADS];
     struct NativeMiniDfsConf conf = {
-        .doFormat = 1,
+        1, /* doFormat */
     };
 
     tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
@@ -323,21 +321,20 @@ int main(void)
         ti[i].threadIdx = i;
     }
 
-    EXPECT_ZERO(sem_init(&tlhSem, 0, tlhNumThreads));
     tlhCluster = nmdCreate(&conf);
     EXPECT_NONNULL(tlhCluster);
     EXPECT_ZERO(nmdWaitClusterUp(tlhCluster));
 
     for (i = 0; i < tlhNumThreads; i++) {
-        EXPECT_ZERO(pthread_create(&ti[i].thread, NULL,
-            testHdfsOperations, &ti[i]));
+        ti[i].theThread.start = testHdfsOperations;
+        ti[i].theThread.arg = &ti[i];
+        EXPECT_ZERO(threadCreate(&ti[i].theThread));
     }
     for (i = 0; i < tlhNumThreads; i++) {
-        EXPECT_ZERO(pthread_join(ti[i].thread, NULL));
+        EXPECT_ZERO(threadJoin(&ti[i].theThread));
     }
 
     EXPECT_ZERO(nmdShutdown(tlhCluster));
     nmdFree(tlhCluster);
-    EXPECT_ZERO(sem_destroy(&tlhSem));
     return checkFailures(ti, tlhNumThreads);
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_native_mini_dfs.c

@@ -22,7 +22,7 @@
 #include <errno.h>
 
 static struct NativeMiniDfsConf conf = {
-    .doFormat = 1,
+    1, /* doFormat */
 };
 
 /**

この差分においてかなりの量のファイルが変更されているため、一部のファイルを表示していません