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

svn commit: r1619016 - in /hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native: common/ fs/ jni/ test/ test/fs/

Author: cmccabe
Date: Wed Aug 20 00:46:20 2014
New Revision: 1619016

URL: http://svn.apache.org/r1619016
Log:
merge HDFS-573 changes from trunk to jnifs.c

Added:
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/platform.h
Modified:
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/test.h
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/fs/fs.c
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.c
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.h
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/jnifs.c
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_threaded.c
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_zerocopy.c
    hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/test.h

Added: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/platform.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/platform.h?rev=1619016&view=auto
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/platform.h (added)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/platform.h Wed Aug 20 00:46:20 2014
@@ -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

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/test.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/test.h?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/test.h (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/common/test.h Wed Aug 20 00:46:20 2014
@@ -120,6 +120,13 @@ void *xcalloc(size_t len);
   EXPECT(expected_buf, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, "%d", x); \
 } while(0);
 
+#define EXPECT_UINT64_EQ(expected, x) do { \
+  char expected_buf[32] = { 0 }; \
+  snprintf(expected_buf, sizeof(expected_buf), "%" PRIu64, expected); \
+  EXPECT(expected_buf, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+         "%" PRIu64, x); \
+} while(0);
+
 #define EXPECT_INT64_EQ(expected, x) do { \
   char expected_buf[32] = { 0 }; \
   snprintf(expected_buf, sizeof(expected_buf), "%" PRId64, expected); \

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/fs/fs.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/fs/fs.c?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/fs/fs.c (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/fs/fs.c Wed Aug 20 00:46:20 2014
@@ -241,7 +241,7 @@ static struct hadoop_err *hdfs_builder_p
     if (hdfs_bld->port == 0) {
         hdfs_bld->port = hdfs_bld->uri->port;
     } else {
-        if (hdfs_bld->port != hdfs_bld->uri->port) {
+        if ((hdfs_bld->port != hdfs_bld->uri->port) && (hdfs_bld->uri->port)) {
             err = hadoop_lerr_alloc(EINVAL, "The connection URI specified "
                     "port %d, but hdfsBuilderSetNameNodePort specified port "
                     "%d.  Please only specify the port once, preferrably in "

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.c?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.c (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.c Wed Aug 20 00:46:20 2014
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
+#include "common/platform.h"
 #include "fs/hdfs.h"
 #include "jni/exception.h"
 #include "jni/jni_helper.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,
     },
 };
 
@@ -114,6 +114,7 @@ int printExceptionAndFreeV(JNIEnv *env, 
     jstring jStr = NULL;
     jvalue jVal;
     jthrowable jthr;
+    const char *stackTrace;
 
     jthr = classNameOfObject(exc, env, &className);
     if (jthr) {
@@ -149,7 +150,7 @@ int printExceptionAndFreeV(JNIEnv *env, 
             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);

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.h?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.h (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/exception.h Wed Aug 20 00:46:20 2014
@@ -34,13 +34,14 @@
  * usually not what you want.)
  */
 
+#include "common/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, 
  *                        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, j
  *                        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(J
  * @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

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/jnifs.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/jnifs.c?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/jnifs.c (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/jni/jnifs.c Wed Aug 20 00:46:20 2014
@@ -61,9 +61,9 @@
  * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
  */
 enum hdfsStreamType {
-    UNINITIALIZED = 0,
-    INPUT = 1,
-    OUTPUT = 2,
+    HDFS_STREAM_UNINITIALIZED = 0,
+    HDFS_STREAM_INPUT = 1,
+    HDFS_STREAM_OUTPUT = 2,
 };
 
 struct jni_fs {
@@ -109,13 +109,13 @@ static jthrowable getFileInfoFromStat(JN
 static int jni_file_is_open_for_read(hdfsFile bfile)
 {
     struct jni_file *file = (struct jni_file*)bfile;
-    return (file->type == INPUT);
+    return (file->type == HDFS_STREAM_INPUT);
 }
 
 static int jni_file_is_open_for_write(hdfsFile bfile)
 {
     struct jni_file *file = (struct jni_file*)bfile;
-    return (file->type == OUTPUT);
+    return (file->type == HDFS_STREAM_OUTPUT);
 }
 
 static int jni_file_get_read_statistics(hdfsFile bfile,
@@ -133,7 +133,7 @@ static int jni_file_get_read_statistics(
         errno = EINTERNAL;
         return -1;
     }
-    if (file->type != INPUT) {
+    if (file->type != HDFS_STREAM_INPUT) {
         ret = EINVAL;
         goto done;
     }
@@ -325,7 +325,7 @@ static int calcEffectiveURI(struct hdfsB
     if (bld->port == 0) {
         suffix[0] = '\0';
     } else {
-        lastColon = rindex(bld->nn, ':');
+        lastColon = strrchr(bld->nn, ':');
         if (lastColon && (strspn(lastColon + 1, "0123456789") ==
                           strlen(lastColon + 1))) {
             fprintf(stderr, "port %d was given, but URI '%s' already "
@@ -611,6 +611,11 @@ static hdfsFile jni_open_file(hdfsFS bfs
     jvalue jVal;
     struct jni_file *file = NULL;
     int ret;
+    /* The hadoop java api/signature */
+    const char* method = NULL;
+    const char* signature = NULL;
+    jint jBufferSize;
+    jshort jReplication;
 
     /*
       JAVA EQUIVALENT:
@@ -643,19 +648,15 @@ static hdfsFile jni_open_file(hdfsFS bfs
       fprintf(stderr, "WARN: hadoop does not truly support O_CREATE && O_EXCL\n");
     }
 
-    /* The hadoop java api/signature */
-    const char* method = NULL;
-    const char* signature = NULL;
-
     if (accmode == O_RDONLY) {
-	method = "open";
-        signature = JMETHOD2(JPARAM(HADOOP_PATH), "I", JPARAM(HADOOP_ISTRM));
+        method = "open";
+            signature = JMETHOD2(JPARAM(HADOOP_PATH), "I", JPARAM(HADOOP_ISTRM));
     } else if (flags & O_APPEND) {
-	method = "append";
-	signature = JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_OSTRM));
+        method = "append";
+        signature = JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_OSTRM));
     } else {
-	method = "create";
-	signature = JMETHOD2(JPARAM(HADOOP_PATH), "ZISJ", JPARAM(HADOOP_OSTRM));
+        method = "create";
+        signature = JMETHOD2(JPARAM(HADOOP_PATH), "ZISJ", JPARAM(HADOOP_OSTRM));
     }
 
     /* Create an object of org.apache.hadoop.fs.Path */
@@ -676,8 +677,8 @@ static hdfsFile jni_open_file(hdfsFS bfs
     }
     jConfiguration = jVal.l;
 
-    jint jBufferSize = bufferSize;
-    jshort jReplication = replication;
+    jBufferSize = bufferSize;
+    jReplication = replication;
     jStrBufferSize = (*env)->NewStringUTF(env, "io.file.buffer.size"); 
     if (!jStrBufferSize) {
         ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, "OOM");
@@ -714,7 +715,7 @@ static hdfsFile jni_open_file(hdfsFS bfs
                     path);
                 goto done;
             }
-            jReplication = jVal.i;
+            jReplication = (jshort)jVal.i;
         }
     }
  
@@ -764,7 +765,8 @@ static hdfsFile jni_open_file(hdfsFS bfs
             "jni_open_file(%s): NewGlobalRef", path); 
         goto done;
     }
-    file->type = (((flags & O_WRONLY) == 0) ? INPUT : OUTPUT);
+    file->type = (((flags & O_WRONLY) == 0) ? HDFS_STREAM_INPUT :
+                  HDFS_STREAM_OUTPUT);
     file->flags = 0;
 
     if ((flags & O_WRONLY) == 0) {
@@ -813,15 +815,15 @@ static int jni_close_file(hdfsFS fs __at
         errno = EINTERNAL;
         return -1;
     }
-    if (file->type == UNINITIALIZED) {
+    if (file->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
-    interface = (file->type == INPUT) ?  HADOOP_ISTRM : HADOOP_OSTRM;
+    interface = (file->type == HDFS_STREAM_INPUT) ?  HADOOP_ISTRM : HADOOP_OSTRM;
     jthr = invokeMethod(env, NULL, INSTANCE, file->stream, interface,
                      "close", "()V");
     if (jthr) {
-        const char *interfaceShortName = (file->type == INPUT) ? 
+        const char *interfaceShortName = (file->type == HDFS_STREAM_INPUT) ? 
             "FSDataInputStream" : "FSDataOutputStream";
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
                 "%s#close", interfaceShortName);
@@ -928,7 +930,7 @@ static tSize jni_read(hdfsFS bfs __attri
     } else if (length < 0) {
         errno = EINVAL;
         return -1;
-    } else if (file->type != INPUT) {
+    } else if (file->type != HDFS_STREAM_INPUT) {
         errno = EINVAL;
         return -1;
     }
@@ -983,7 +985,7 @@ static tSize jni_pread(hdfsFS bfs __attr
         errno = EINVAL;
         return -1;
     }
-    if (file->type == UNINITIALIZED) {
+    if (file->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
@@ -995,7 +997,7 @@ static tSize jni_pread(hdfsFS bfs __attr
     }
 
     //Error checking... make sure that this file is 'readable'
-    if (file->type != INPUT) {
+    if (file->type != HDFS_STREAM_INPUT) {
         fprintf(stderr, "Cannot read from a non-InputStream object!\n");
         errno = EINVAL;
         return -1;
@@ -1055,7 +1057,7 @@ static tSize jni_write(hdfsFS bfs __attr
     }
 
     //Sanity check
-    if (file->type == UNINITIALIZED) {
+    if (file->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
@@ -1064,7 +1066,7 @@ static tSize jni_write(hdfsFS bfs __attr
     	return -1;
     }
     //Error checking... make sure that this file is 'writable'
-    if (file->type != OUTPUT) {
+    if (file->type != HDFS_STREAM_OUTPUT) {
         fprintf(stderr, "Cannot write into a non-OutputStream object!\n");
         errno = EINVAL;
         return -1;
@@ -1116,7 +1118,7 @@ static int jni_seek(hdfsFS bfs __attribu
       errno = EINTERNAL;
       return -1;
     }
-    if (file->type != INPUT) {
+    if (file->type != HDFS_STREAM_INPUT) {
         errno = EBADF;
         return -1;
     }
@@ -1145,13 +1147,13 @@ static tOffset jni_tell(hdfsFS bfs __att
     }
 
     //Sanity check
-    if (file->type == UNINITIALIZED) {
+    if (file->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
 
     //Parameters
-    const char* interface = (file->type == INPUT) ?
+    const char* interface = (file->type == HDFS_STREAM_INPUT) ?
         HADOOP_ISTRM : HADOOP_OSTRM;
     jvalue jVal;
     jthrowable jthr = invokeMethod(env, &jVal, INSTANCE, file->stream,
@@ -1159,7 +1161,7 @@ static tOffset jni_tell(hdfsFS bfs __att
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
             "jni_tell: %s#getPos",
-            ((file->type == INPUT) ? "FSDataInputStream" :
+            ((file->type == HDFS_STREAM_INPUT) ? "FSDataInputStream" :
                                  "FSDataOutputStream"));
         return -1;
     }
@@ -1176,7 +1178,7 @@ static int jni_flush(hdfsFS bfs __attrib
       errno = EINTERNAL;
       return -1;
     }
-    if (file->type != OUTPUT) {
+    if (file->type != HDFS_STREAM_OUTPUT) {
         errno = EBADF;
         return -1;
     }
@@ -1200,7 +1202,7 @@ static int jni_hflush(hdfsFS bfs __attri
       errno = EINTERNAL;
       return -1;
     }
-    if (file->type != OUTPUT) {
+    if (file->type != HDFS_STREAM_OUTPUT) {
         errno = EBADF;
         return -1;
     }
@@ -1224,7 +1226,7 @@ static int jni_hsync(hdfsFS bfs __attrib
       errno = EINTERNAL;
       return -1;
     }
-    if (file->type != OUTPUT) {
+    if (file->type != HDFS_STREAM_OUTPUT) {
         errno = EBADF;
         return -1;
     }
@@ -1250,7 +1252,7 @@ static int jni_available(hdfsFS bfs __at
       errno = EINTERNAL;
       return -1;
     }
-    if (file->type != INPUT) {
+    if (file->type != HDFS_STREAM_INPUT) {
         errno = EBADF;
         return -1;
     }
@@ -1397,6 +1399,11 @@ static int jni_rename(hdfsFS bfs, const 
     //  Path new = new Path(newPath);
     //  fs.rename(old, new);
 
+    jthrowable jthr;
+    jobject jOldPath = NULL, jNewPath = NULL;
+    int ret = -1;
+    jvalue jVal;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1404,11 +1411,6 @@ static int jni_rename(hdfsFS bfs, const 
       return -1;
     }
 
-    jthrowable jthr;
-    jobject jOldPath = NULL, jNewPath = NULL;
-    int ret = -1;
-    jvalue jVal;
-
     jthr = constructNewObjectOfPath(env, oldPath, &jOldPath );
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1449,6 +1451,13 @@ static char* jni_get_working_directory(h
                                        size_t bufferSize)
 {
     struct jni_fs *fs = (struct jni_fs*)bfs;
+    jobject jPath = NULL;
+    jstring jPathString = NULL;
+    jvalue jVal;
+    jthrowable jthr;
+    int ret;
+    const char *jPathChars = NULL;
+
     // JAVA EQUIVALENT:
     //  Path p = fs.getWorkingDirectory(); 
     //  return p.toString()
@@ -1460,13 +1469,6 @@ static char* jni_get_working_directory(h
       return NULL;
     }
 
-    jobject jPath = NULL;
-    jstring jPathString = NULL;
-    jvalue jVal;
-    jthrowable jthr;
-    int ret;
-    const char *jPathChars = NULL;
-
     //FileSystem#getWorkingDirectory()
     jthr = invokeMethod(env, &jVal, INSTANCE, fs->obj,
                      HADOOP_FS, "getWorkingDirectory",
@@ -1530,6 +1532,9 @@ static int jni_set_working_directory(hdf
     // JAVA EQUIVALENT:
     //  fs.setWorkingDirectory(Path(path)); 
 
+    jthrowable jthr;
+    jobject jPath;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1537,9 +1542,6 @@ static int jni_set_working_directory(hdf
       return -1;
     }
 
-    jthrowable jthr;
-    jobject jPath;
-
     //Create an object of org.apache.hadoop.fs.Path
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
@@ -1569,6 +1571,9 @@ static int jni_mkdir(hdfsFS bfs, const c
     // JAVA EQUIVALENT:
     //  fs.mkdirs(new Path(path));
 
+    jobject jPath;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1576,9 +1581,6 @@ static int jni_mkdir(hdfsFS bfs, const c
       return -1;
     }
 
-    jobject jPath;
-    jthrowable jthr;
-
     //Create an object of org.apache.hadoop.fs.Path
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
@@ -1619,7 +1621,8 @@ static int jni_set_replication(hdfsFS bf
     struct jni_fs *fs = (struct jni_fs*)bfs;
     // JAVA EQUIVALENT:
     //  fs.setReplication(new Path(path), replication);
-
+    jthrowable jthr;
+    jobject jPath;
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1627,10 +1630,6 @@ static int jni_set_replication(hdfsFS bf
       return -1;
     }
 
-    jthrowable jthr;
-
-    //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath;
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -2040,7 +2039,7 @@ static struct hadoopRzBuffer* jni_read_z
         errno = EINTERNAL;
         return NULL;
     }
-    if (file->type != INPUT) {
+    if (file->type != HDFS_STREAM_INPUT) {
         fputs("Cannot read from a non-InputStream object!\n", stderr);
         ret = EINVAL;
         goto done;
@@ -2364,7 +2363,8 @@ static tOffset jni_get_capacity(hdfsFS b
     //  FsStatus fss = fs.getStatus();
     //  return Fss.getCapacity();
 
-    //Get the JNIEnv* corresponding to current thread
+    jvalue  jVal;
+    jthrowable jthr;
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
       errno = EINTERNAL;
@@ -2372,8 +2372,6 @@ static tOffset jni_get_capacity(hdfsFS b
     }
 
     //FileSystem#getStatus
-    jvalue  jVal;
-    jthrowable jthr;
     jthr = invokeMethod(env, &jVal, INSTANCE, fs->obj, HADOOP_FS,
                      "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
     if (jthr) {
@@ -2575,7 +2573,8 @@ static hdfsFileInfo *jni_get_path_info(h
     //  fs.getLength(f)
     //  f.getPath()
 
-    //Get the JNIEnv* corresponding to current thread
+    jobject jPath;
+    jthrowable jthr;
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
       errno = EINTERNAL;
@@ -2583,8 +2582,7 @@ static hdfsFileInfo *jni_get_path_info(h
     }
 
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath;
-    jthrowable jthr = constructNewObjectOfPath(env, path, &jPath);
+    jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
             "jni_get_path_info(%s): constructNewObjectOfPath", path);
@@ -2614,7 +2612,8 @@ static tOffset jni_get_used(hdfsFS bfs)
     //  FsStatus fss = fs.getStatus();
     //  return Fss.getUsed();
 
-    //Get the JNIEnv* corresponding to current thread
+    jvalue  jVal;
+    jthrowable jthr;
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
       errno = EINTERNAL;
@@ -2622,8 +2621,6 @@ static tOffset jni_get_used(hdfsFS bfs)
     }
 
     //FileSystem#getStatus
-    jvalue  jVal;
-    jthrowable jthr;
     jthr = invokeMethod(env, &jVal, INSTANCE, fs->obj, HADOOP_FS,
                      "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
     if (jthr) {
@@ -2648,8 +2645,10 @@ int jni_chown(hdfsFS bfs, const char* pa
     struct jni_fs *fs = (struct jni_fs*)bfs;
     // JAVA EQUIVALENT:
     //  fs.setOwner(path, owner, group)
-
-    //Get the JNIEnv* corresponding to current thread
+    jobject jPath = NULL;
+    jstring jOwner = NULL, jGroup = NULL;
+    jthrowable jthr;
+    int ret;
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
       errno = EINTERNAL;
@@ -2660,11 +2659,6 @@ int jni_chown(hdfsFS bfs, const char* pa
       return 0;
     }
 
-    jobject jPath = NULL;
-    jstring jOwner = NULL, jGroup = NULL;
-    jthrowable jthr;
-    int ret;
-
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -2719,18 +2713,16 @@ int jni_chmod(hdfsFS bfs, const char* pa
     // JAVA EQUIVALENT:
     //  fs.setPermission(path, FsPermission)
 
-    //Get the JNIEnv* corresponding to current thread
+    jthrowable jthr;
+    jobject jPath = NULL, jPermObj = NULL;
+    jshort jmode = mode;
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
       errno = EINTERNAL;
       return -1;
     }
 
-    jthrowable jthr;
-    jobject jPath = NULL, jPermObj = NULL;
-
     // construct jPerm = FsPermission.createImmutable(short mode);
-    jshort jmode = mode;
     jthr = constructNewObjectOfClass(env, &jPermObj,
                 HADOOP_FSPERM,"(S)V",jmode);
     if (jthr) {
@@ -2779,27 +2771,21 @@ static int jni_utime(hdfsFS bfs, const c
     // JAVA EQUIVALENT:
     //  fs.setTimes(src, mtime, atime)
     jthrowable jthr;
-
-    //Get the JNIEnv* corresponding to current thread
+    jobject jPath;
+    const tTime NO_CHANGE = -1;
+    jlong jmtime = (mtime == NO_CHANGE) ? -1 : (mtime * (jlong)1000);
+    jlong jatime = (atime == NO_CHANGE) ? -1 : (atime * (jlong)1000);
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
       errno = EINTERNAL;
       return -1;
     }
-
-    //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath;
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
             "jni_utime(path=%s): constructNewObjectOfPath", path);
         return -1;
     }
-
-    const tTime NO_CHANGE = -1;
-    jlong jmtime = (mtime == NO_CHANGE) ? -1 : (mtime * (jlong)1000);
-    jlong jatime = (atime == NO_CHANGE) ? -1 : (atime * (jlong)1000);
-
     jthr = invokeMethod(env, NULL, INSTANCE, fs->obj, HADOOP_FS,
             "setTimes", JMETHOD3(JPARAM(HADOOP_PATH), "J", "J", JAVA_VOID),
             jPath, jmtime, jatime);

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_threaded.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_threaded.c?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_threaded.c (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_threaded.c Wed Aug 20 00:46:20 2014
@@ -16,17 +16,18 @@
  * limitations under the License.
  */
 
+#include "common/platform.h"
 #include "fs/hdfs.h"
 #include "test/native_mini_dfs.h"
 #include "test/test.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>
+#include <uv.h>
 
 #define TO_STR_HELPER(X) #X
 #define TO_STR(X) TO_STR_HELPER(X)
@@ -35,8 +36,6 @@
 
 #define TLH_DEFAULT_BLOCK_SIZE 134217728
 
-static sem_t tlhSem;
-
 static struct NativeMiniDfsCluster* tlhCluster;
 
 struct tlhThreadInfo {
@@ -44,18 +43,19 @@ struct tlhThreadInfo {
     int threadIdx;
     /** 0 = thread was successful; error code otherwise */
     int success;
-    /** pthread identifier */
-    pthread_t thread;
+    /** thread identifier */
+    uv_thread_t theThread;
 };
 
 static int hdfsSingleNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs,
                                      const char *username)
 {
-    int ret, port;
+    int ret;
+    int 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);
@@ -85,7 +85,7 @@ static int hdfsSingleNameNodeConnect(str
 
 static int doTestGetDefaultBlockSize(hdfsFS fs, const char *path)
 {
-    int64_t blockSize;
+    tOffset blockSize;
     int ret;
 
     blockSize = hdfsGetDefaultBlockSize(fs);
@@ -122,23 +122,24 @@ struct tlhPaths {
 
 static int setupPaths(const struct tlhThreadInfo *ti, struct tlhPaths *paths)
 {
-    memset(paths, sizeof(*paths), 0);
-    if ((size_t)snprintf(paths->prefix, sizeof(paths->prefix), "/tlhData%04d",
-                 ti->threadIdx) >= sizeof(paths->prefix)) {
+    memset(paths, 0, sizeof(*paths));
+    if (snprintf(paths->prefix, sizeof(paths->prefix), "/tlhData%04d",
+                 ti->threadIdx) >= (int)sizeof(paths->prefix)) {
         return ENAMETOOLONG;
     }
-    if ((size_t)snprintf(paths->file1, sizeof(paths->file1), "%s/file1",
-                 paths->prefix) >= sizeof(paths->file1)) {
+    if (snprintf(paths->file1, sizeof(paths->file1), "%s/file1",
+                 paths->prefix) >= (int)sizeof(paths->file1)) {
         return ENAMETOOLONG;
     }
-    if ((size_t)snprintf(paths->file2, sizeof(paths->file2), "%s/file2",
-                 paths->prefix) >= sizeof(paths->file2)) {
+    if (snprintf(paths->file2, sizeof(paths->file2), "%s/file2",
+                 paths->prefix) >= (int)sizeof(paths->file2)) {
         return ENAMETOOLONG;
     }
     return 0;
 }
 
-static int doTestHdfsOperations(hdfsFS fs, const struct tlhPaths *paths)
+static int doTestHdfsOperations(hdfsFS fs,
+                                const struct tlhPaths *paths)
 {
     char tmp[4096];
     hdfsFile file;
@@ -163,7 +164,7 @@ static int doTestHdfsOperations(hdfsFS f
     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;
@@ -185,9 +186,9 @@ static int doTestHdfsOperations(hdfsFS f
 
     EXPECT_INT_ZERO(hdfsFileGetReadStatistics(file, &readStats));
     errno = 0;
-    EXPECT_INT_ZERO(readStats->totalBytesRead);
-    EXPECT_INT_ZERO(readStats->totalLocalBytesRead);
-    EXPECT_INT_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));
@@ -203,7 +204,7 @@ static int doTestHdfsOperations(hdfsFS f
     }
     EXPECT_INT_ZERO(hdfsFileGetReadStatistics(file, &readStats));
     errno = 0;
-    EXPECT_INT_EQ(expected, readStats->totalBytesRead);
+    EXPECT_UINT64_EQ((uint64_t)expected, readStats->totalBytesRead);
     hdfsFileFreeReadStatistics(readStats);
     EXPECT_INT_ZERO(memcmp(paths->prefix, tmp, expected));
     EXPECT_INT_ZERO(hdfsCloseFile(fs, file));
@@ -261,12 +262,11 @@ static int testHdfsOperationsImpl(struct
     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)
@@ -303,7 +303,10 @@ int main(void)
     const char *tlhNumThreadsStr;
     struct tlhThreadInfo ti[TLH_MAX_THREADS];
     struct NativeMiniDfsConf conf = {
-        .doFormat = 1,
+        1, /* doFormat */
+        0, /* webHdfsEnabled */
+        0, /* nameNodeHttpPort */
+        0, /* configureShortCircuit */
     };
 
     tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
@@ -322,21 +325,19 @@ int main(void)
         ti[i].threadIdx = i;
     }
 
-    EXPECT_INT_ZERO(sem_init(&tlhSem, 0, tlhNumThreads));
     tlhCluster = nmdCreate(&conf);
     EXPECT_NONNULL(tlhCluster);
     EXPECT_INT_ZERO(nmdWaitClusterUp(tlhCluster));
 
     for (i = 0; i < tlhNumThreads; i++) {
-        EXPECT_INT_ZERO(pthread_create(&ti[i].thread, NULL,
-            testHdfsOperations, &ti[i]));
+        EXPECT_INT_ZERO(uv_thread_create(&ti[i].theThread,
+                                     testHdfsOperations, &ti[i]));
     }
     for (i = 0; i < tlhNumThreads; i++) {
-        EXPECT_INT_ZERO(pthread_join(ti[i].thread, NULL));
+        EXPECT_INT_ZERO(uv_thread_join(&ti[i].theThread));
     }
 
     EXPECT_INT_ZERO(nmdShutdown(tlhCluster));
     nmdFree(tlhCluster);
-    EXPECT_INT_ZERO(sem_destroy(&tlhSem));
     return checkFailures(ti, tlhNumThreads);
 }

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_zerocopy.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_zerocopy.c?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_zerocopy.c (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/fs/test_libhdfs_zerocopy.c Wed Aug 20 00:46:20 2014
@@ -94,18 +94,18 @@ static 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);

Modified: hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/test.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/test.h?rev=1619016&r1=1619015&r2=1619016&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/test.h (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-native-core/src/main/native/test/test.h Wed Aug 20 00:46:20 2014
@@ -121,6 +121,13 @@ void *xcalloc(size_t len);
   EXPECT(expected_buf, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, "%d", x); \
 } while(0);
 
+#define EXPECT_UINT64_EQ(expected, x) do { \
+  char expected_buf[32] = { 0 }; \
+  snprintf(expected_buf, sizeof(expected_buf), "%" PRIu64, expected); \
+  EXPECT(expected_buf, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+         "%" PRIu64, x); \
+} while(0);
+
 #define EXPECT_INT64_EQ(expected, x) do { \
   char expected_buf[32] = { 0 }; \
   snprintf(expected_buf, sizeof(expected_buf), "%" PRId64, expected); \