You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by at...@apache.org on 2012/08/06 23:02:31 UTC

svn commit: r1370015 [2/3] - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/ src/main/native/libhdfs/

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c?rev=1370015&r1=1370014&r2=1370015&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c Mon Aug  6 21:02:30 2012
@@ -16,9 +16,11 @@
  * limitations under the License.
  */
 
+#include "exception.h"
 #include "hdfs.h"
 #include "jni_helper.h"
 
+#include <inttypes.h>
 #include <stdio.h>
 #include <string.h>
 
@@ -53,6 +55,7 @@
 #define HDFS_FILE_SUPPORTS_DIRECT_READ (1<<0)
 
 tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length);
+static void hdfsFreeFileInfoEntry(hdfsFileInfo *hdfsFileInfo);
 
 /**
  * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
@@ -72,7 +75,7 @@ struct hdfsFile_internal {
     enum hdfsStreamType type;
     int flags;
 };
-    
+
 int hdfsFileIsOpenForRead(hdfsFile file)
 {
     return (file->type == INPUT);
@@ -102,21 +105,6 @@ typedef struct
     JNIEnv* env;
 } hdfsJniEnv;
 
-
-
-/**
- * Helper function to destroy a local reference of java.lang.Object
- * @param env: The JNIEnv pointer. 
- * @param jFile: The local reference of java.lang.Object object
- * @return None.
- */
-static void destroyLocalReference(JNIEnv *env, jobject jObject)
-{
-  if (jObject)
-    (*env)->DeleteLocalRef(env, jObject);
-}
-
-
 /**
  * Helper function to create a org.apache.hadoop.fs.Path object.
  * @param env: The JNIEnv pointer. 
@@ -124,95 +112,25 @@ static void destroyLocalReference(JNIEnv
  * object.
  * @return Returns a jobject on success and NULL on error.
  */
-static jobject constructNewObjectOfPath(JNIEnv *env, const char *path)
+static jthrowable constructNewObjectOfPath(JNIEnv *env, const char *path,
+                                           jobject *out)
 {
-    //Construct a java.lang.String object
-    jstring jPathString = (*env)->NewStringUTF(env, path); 
+    jthrowable jthr;
+    jstring jPathString;
+    jobject jPath;
 
+    //Construct a java.lang.String object
+    jthr = newJavaStr(env, path, &jPathString);
+    if (jthr)
+        return jthr;
     //Construct the org.apache.hadoop.fs.Path object
-    jobject jPath =
-        constructNewObjectOfClass(env, NULL, "org/apache/hadoop/fs/Path",
-                                  "(Ljava/lang/String;)V", jPathString);
-    if (jPath == NULL) {
-        fprintf(stderr, "Can't construct instance of class "
-                "org.apache.hadoop.fs.Path for %s\n", path);
-        errno = EINTERNAL;
-        return NULL;
-    }
-
-    // Destroy the local reference to the java.lang.String object
+    jthr = constructNewObjectOfClass(env, &jPath, "org/apache/hadoop/fs/Path",
+                                     "(Ljava/lang/String;)V", jPathString);
     destroyLocalReference(env, jPathString);
-
-    return jPath;
-}
-
-
-/**
- * Helper function to translate an exception into a meaningful errno value.
- * @param exc: The exception.
- * @param env: The JNIEnv Pointer.
- * @param method: The name of the method that threw the exception. This
- * may be format string to be used in conjuction with additional arguments.
- * @return Returns a meaningful errno value if possible, or EINTERNAL if not.
- */
-static int errnoFromException(jthrowable exc, JNIEnv *env,
-                              const char *method, ...)
-{
-    va_list ap;
-    int errnum = 0;
-    char *excClass = NULL;
-
-    if (exc == NULL)
-        goto default_error;
-
-    if ((excClass = classNameOfObject((jobject) exc, env)) == NULL) {
-      errnum = EINTERNAL;
-      goto done;
-    }
-
-    if (!strcmp(excClass, "java.lang.UnsupportedOperationException")) {
-      errnum = ENOTSUP;
-      goto done;
-    }
-
-    if (!strcmp(excClass, "org.apache.hadoop.security."
-                "AccessControlException")) {
-        errnum = EACCES;
-        goto done;
-    }
-
-    if (!strcmp(excClass, "org.apache.hadoop.hdfs.protocol."
-                "QuotaExceededException")) {
-        errnum = EDQUOT;
-        goto done;
-    }
-
-    if (!strcmp(excClass, "java.io.FileNotFoundException")) {
-        errnum = ENOENT;
-        goto done;
-    }
-
-    //TODO: interpret more exceptions; maybe examine exc.getMessage()
-
-default_error:
-
-    //Can't tell what went wrong, so just punt
-    (*env)->ExceptionDescribe(env);
-    fprintf(stderr, "Call to ");
-    va_start(ap, method);
-    vfprintf(stderr, method, ap);
-    va_end(ap);
-    fprintf(stderr, " failed!\n");
-    errnum = EINTERNAL;
-
-done:
-
-    (*env)->ExceptionClear(env);
-
-    if (excClass != NULL)
-        free(excClass);
-
-    return errnum;
+    if (jthr)
+        return jthr;
+    *out = jPath;
+    return NULL;
 }
 
 /**
@@ -223,123 +141,80 @@ done:
  * @param key               The key to modify
  * @param value             The value to set the key to
  *
- * @return                  0 on success; error code otherwise
+ * @return                  NULL on success; exception otherwise
  */
-static int hadoopConfSetStr(JNIEnv *env, jobject jConfiguration,
+static jthrowable hadoopConfSetStr(JNIEnv *env, jobject jConfiguration,
         const char *key, const char *value)
 {
-    int ret;
-    jthrowable jExc = NULL;
+    jthrowable jthr;
     jstring jkey = NULL, jvalue = NULL;
-    char buf[1024];
 
-    jkey = (*env)->NewStringUTF(env, key);
-    if (!jkey) {
-        ret = ENOMEM;
+    jthr = newJavaStr(env, key, &jkey);
+    if (jthr)
         goto done;
-    }
-    jvalue = (*env)->NewStringUTF(env, value);
-    if (!jvalue) {
-        ret = ENOMEM;
+    jthr = newJavaStr(env, value, &jvalue);
+    if (jthr)
         goto done;
-    }
-    ret = invokeMethod(env, NULL, &jExc, INSTANCE, jConfiguration,
+    jthr = invokeMethod(env, NULL, INSTANCE, jConfiguration,
             HADOOP_CONF, "set", JMETHOD2(JPARAM(JAVA_STRING),
                                          JPARAM(JAVA_STRING), JAVA_VOID),
             jkey, jvalue);
-    if (ret) {
-        snprintf(buf, sizeof(buf), "hadoopConfSet(%s, %s)", key, value);
-        ret = errnoFromException(jExc, env, buf);
+    if (jthr)
         goto done;
-    }
 done:
-    if (jkey)
-        destroyLocalReference(env, jkey);
-    if (jvalue)
-        destroyLocalReference(env, jvalue);
-    if (ret)
-        errno = ret;
-    return ret;
-}
-
-/**
- * Convert a Java string into a C string.
- *
- * @param env               The JNI environment
- * @param jStr              The Java string to convert
- * @param cstr              (out param) the C string.
- *                          This will be set to a dynamically allocated
- *                          UTF-8 C string on success.
- *
- * @return                  0 on success; error code otherwise
- */
-static int jStrToCstr(JNIEnv *env, jstring jstr, char **cstr)
-{
-    const char *tmp;
-
-    tmp = (*env)->GetStringUTFChars(env, jstr, NULL);
-    *cstr = strdup(tmp);
-    (*env)->ReleaseStringUTFChars(env, jstr, tmp);
-    return 0;
+    destroyLocalReference(env, jkey);
+    destroyLocalReference(env, jvalue);
+    return jthr;
 }
 
-static int hadoopConfGetStr(JNIEnv *env, jobject jConfiguration,
+static jthrowable hadoopConfGetStr(JNIEnv *env, jobject jConfiguration,
         const char *key, char **val)
 {
-    int ret;
-    jthrowable jExc = NULL;
+    jthrowable jthr;
     jvalue jVal;
-    jstring jkey = NULL;
-    char buf[1024];
+    jstring jkey = NULL, jRet = NULL;
 
-    jkey = (*env)->NewStringUTF(env, key);
-    if (!jkey) {
-        ret = ENOMEM;
+    jthr = newJavaStr(env, key, &jkey);
+    if (jthr)
         goto done;
-    }
-    ret = invokeMethod(env, &jVal, &jExc, INSTANCE, jConfiguration,
+    jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
             HADOOP_CONF, "get", JMETHOD1(JPARAM(JAVA_STRING),
                                          JPARAM(JAVA_STRING)), jkey);
-    if (ret) {
-        snprintf(buf, sizeof(buf), "hadoopConfGetStr(%s)", key);
-        ret = errnoFromException(jExc, env, buf);
-        goto done;
-    }
-    if (!jVal.l) {
-        *val = NULL;
-        goto done;
-    }
-
-    ret = jStrToCstr(env, jVal.l, val);
-    if (ret)
+    if (jthr)
         goto done;
+    jRet = jVal.l;
+    jthr = newCStr(env, jRet, val);
 done:
-    if (jkey)
-        destroyLocalReference(env, jkey);
-    if (ret)
-        errno = ret;
-    return ret;
+    destroyLocalReference(env, jkey);
+    destroyLocalReference(env, jRet);
+    return jthr;
 }
 
 int hdfsConfGetStr(const char *key, char **val)
 {
     JNIEnv *env;
     int ret;
+    jthrowable jthr;
     jobject jConfiguration = NULL;
 
     env = getJNIEnv();
     if (env == NULL) {
-      ret = EINTERNAL;
-      goto done;
-    }
-    jConfiguration = constructNewObjectOfClass(env, NULL, HADOOP_CONF, "()V");
-    if (jConfiguration == NULL) {
-        fprintf(stderr, "Can't construct instance of class "
-                "org.apache.hadoop.conf.Configuration\n");
         ret = EINTERNAL;
         goto done;
     }
-    ret = hadoopConfGetStr(env, jConfiguration, key, val);
+    jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsConfGetStr(%s): new Configuration", key);
+        goto done;
+    }
+    jthr = hadoopConfGetStr(env, jConfiguration, key, val);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsConfGetStr(%s): hadoopConfGetStr", key);
+        goto done;
+    }
+    ret = 0;
 done:
     destroyLocalReference(env, jConfiguration);
     if (ret)
@@ -347,30 +222,29 @@ done:
     return ret;
 }
 
-static int hadoopConfGetInt(JNIEnv *env, jobject jConfiguration,
+void hdfsConfStrFree(char *val)
+{
+    free(val);
+}
+
+static jthrowable hadoopConfGetInt(JNIEnv *env, jobject jConfiguration,
         const char *key, int32_t *val)
 {
-    int ret;
-    jthrowable jExc = NULL;
+    jthrowable jthr = NULL;
     jvalue jVal;
     jstring jkey = NULL;
-    char buf[1024];
 
-    jkey = (*env)->NewStringUTF(env, key);
-    if (!jkey) {
-        (*env)->ExceptionDescribe(env);
-        return ENOMEM;
-    }
-    ret = invokeMethod(env, &jVal, &jExc, INSTANCE, jConfiguration,
+    jthr = newJavaStr(env, key, &jkey);
+    if (jthr)
+        return jthr;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
             HADOOP_CONF, "getInt", JMETHOD2(JPARAM(JAVA_STRING), "I", "I"),
             jkey, (jint)(*val));
     destroyLocalReference(env, jkey);
-    if (ret) {
-        snprintf(buf, sizeof(buf), "hadoopConfGetInt(%s)", key);
-        return errnoFromException(jExc, env, buf);
-    }
+    if (jthr)
+        return jthr;
     *val = jVal.i;
-    return 0;
+    return NULL;
 }
 
 int hdfsConfGetInt(const char *key, int32_t *val)
@@ -378,20 +252,26 @@ int hdfsConfGetInt(const char *key, int3
     JNIEnv *env;
     int ret;
     jobject jConfiguration = NULL;
+    jthrowable jthr;
 
     env = getJNIEnv();
     if (env == NULL) {
       ret = EINTERNAL;
       goto done;
     }
-    jConfiguration = constructNewObjectOfClass(env, NULL, HADOOP_CONF, "()V");
-    if (jConfiguration == NULL) {
-        fprintf(stderr, "Can't construct instance of class "
-                "org.apache.hadoop.conf.Configuration\n");
-        ret = EINTERNAL;
+    jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsConfGetInt(%s): new Configuration", key);
+        goto done;
+    }
+    jthr = hadoopConfGetInt(env, jConfiguration, key, val);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsConfGetInt(%s): hadoopConfGetInt", key);
         goto done;
     }
-    ret = hadoopConfGetInt(env, jConfiguration, key, val);
+    ret = 0;
 done:
     destroyLocalReference(env, jConfiguration);
     if (ret)
@@ -399,11 +279,6 @@ done:
     return ret;
 }
 
-void hdfsConfStrFree(char *val)
-{
-    free(val);
-}
-
 struct hdfsBuilder {
     int forceNewInstance;
     const char *nn;
@@ -551,54 +426,72 @@ static int calcEffectiveURI(struct hdfsB
     return 0;
 }
 
+static const char *maybeNull(const char *str)
+{
+    return str ? str : "(NULL)";
+}
+
+static const char *hdfsBuilderToStr(const struct hdfsBuilder *bld,
+                                    char *buf, size_t bufLen)
+{
+    snprintf(buf, bufLen, "forceNewInstance=%d, nn=%s, port=%d, "
+             "kerbTicketCachePath=%s, userName=%s",
+             bld->forceNewInstance, maybeNull(bld->nn), bld->port,
+             maybeNull(bld->kerbTicketCachePath), maybeNull(bld->userName));
+    return buf;
+}
+
 hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld)
 {
     JNIEnv *env = 0;
-    jobject gFsRef = NULL;
     jobject jConfiguration = NULL, jFS = NULL, jURI = NULL, jCachePath = NULL;
     jstring jURIString = NULL, jUserString = NULL;
     jvalue  jVal;
-    jthrowable jExc = NULL;
-    char *cURI = 0;
-    int ret = 0;
+    jthrowable jthr = NULL;
+    char *cURI = 0, buf[512];
+    int ret;
+    jobject jRet = NULL;
 
     //Get the JNIEnv* corresponding to current thread
     env = getJNIEnv();
     if (env == NULL) {
-      ret = EINTERNAL;
-      goto done;
+        ret = EINTERNAL;
+        goto done;
     }
 
     //  jConfiguration = new Configuration();
-    jConfiguration = constructNewObjectOfClass(env, NULL, HADOOP_CONF, "()V");
-    if (jConfiguration == NULL) {
-        fprintf(stderr, "Can't construct instance of class "
-                "org.apache.hadoop.conf.Configuration\n");
-      errno = EINTERNAL;
-      goto done;
+    jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsBuilderConnect(%s)", hdfsBuilderToStr(bld, buf, sizeof(buf)));
+        goto done;
     }
  
     //Check what type of FileSystem the caller wants...
     if (bld->nn == NULL) {
         // Get a local filesystem.
         if (bld->forceNewInstance) {
-            // fs = FileSytem::newInstanceLocal(conf);
-            if (invokeMethod(env, &jVal, &jExc, STATIC, NULL, HADOOP_FS,
+            // fs = FileSytem#newInstanceLocal(conf);
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
                     "newInstanceLocal", JMETHOD1(JPARAM(HADOOP_CONF),
-                    JPARAM(HADOOP_LOCALFS)), jConfiguration)) {
-                ret = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                           "FileSystem::newInstanceLocal");
+                    JPARAM(HADOOP_LOCALFS)), jConfiguration);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
                 goto done;
             }
             jFS = jVal.l;
         } else {
-            // fs = FileSytem::getLocal(conf);
-            if (invokeMethod(env, &jVal, &jExc, STATIC, NULL, HADOOP_FS, "getLocal",
+            // fs = FileSytem#getLocal(conf);
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS, "getLocal",
                              JMETHOD1(JPARAM(HADOOP_CONF),
                                       JPARAM(HADOOP_LOCALFS)),
-                             jConfiguration) != 0) {
-                ret = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                           "FileSystem::getLocal");
+                             jConfiguration);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
                 goto done;
             }
             jFS = jVal.l;
@@ -606,69 +499,95 @@ hdfsFS hdfsBuilderConnect(struct hdfsBui
     } else {
         if (!strcmp(bld->nn, "default")) {
             // jURI = FileSystem.getDefaultUri(conf)
-            if (invokeMethod(env, &jVal, &jExc, STATIC, NULL, HADOOP_FS,
-                          "getDefaultUri", 
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
+                          "getDefaultUri",
                           "(Lorg/apache/hadoop/conf/Configuration;)Ljava/net/URI;",
-                          jConfiguration) != 0) {
-                ret = errnoFromException(jExc, env, "org.apache.hadoop.fs.", 
-                                           "FileSystem::getDefaultUri");
+                          jConfiguration);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
                 goto done;
             }
             jURI = jVal.l;
         } else {
-            // fs = FileSystem::get(URI, conf, ugi);
+            // fs = FileSystem#get(URI, conf, ugi);
             ret = calcEffectiveURI(bld, &cURI);
             if (ret)
                 goto done;
-            jURIString = (*env)->NewStringUTF(env, cURI);
-            if (invokeMethod(env, &jVal, &jExc, STATIC, NULL, JAVA_NET_URI,
+            jthr = newJavaStr(env, cURI, &jURIString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, JAVA_NET_URI,
                              "create", "(Ljava/lang/String;)Ljava/net/URI;",
-                             jURIString) != 0) {
-                ret = errnoFromException(jExc, env, "java.net.URI::create");
+                             jURIString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
                 goto done;
             }
             jURI = jVal.l;
         }
 
         if (bld->kerbTicketCachePath) {
-            ret = hadoopConfSetStr(env, jConfiguration,
+            jthr = hadoopConfSetStr(env, jConfiguration,
                 KERBEROS_TICKET_CACHE_PATH, bld->kerbTicketCachePath);
-            if (ret)
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
                 goto done;
+            }
         }
-        if (bld->userName) {
-            jUserString = (*env)->NewStringUTF(env, bld->userName);
+        jthr = newJavaStr(env, bld->userName, &jUserString);
+        if (jthr) {
+            ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                "hdfsBuilderConnect(%s)",
+                hdfsBuilderToStr(bld, buf, sizeof(buf)));
+            goto done;
         }
         if (bld->forceNewInstance) {
-            if (invokeMethod(env, &jVal, &jExc, STATIC, NULL,
-                    HADOOP_FS, "newInstance",
-                    JMETHOD3(JPARAM(JAVA_NET_URI), JPARAM(HADOOP_CONF),
-                        JPARAM(JAVA_STRING), JPARAM(HADOOP_FS)), jURI,
-                    jConfiguration, jUserString)) {
-                ret = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                           "Filesystem::newInstance(URI, Configuration)");
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
+                    "newInstance", JMETHOD3(JPARAM(JAVA_NET_URI), 
+                        JPARAM(HADOOP_CONF), JPARAM(JAVA_STRING),
+                        JPARAM(HADOOP_FS)),
+                    jURI, jConfiguration, jUserString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
                 goto done;
             }
             jFS = jVal.l;
         } else {
-            if (invokeMethod(env, &jVal, &jExc, STATIC, NULL, HADOOP_FS, "get",
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS, "get",
                     JMETHOD3(JPARAM(JAVA_NET_URI), JPARAM(HADOOP_CONF),
                         JPARAM(JAVA_STRING), JPARAM(HADOOP_FS)),
-                        jURI, jConfiguration, jUserString)) {
-                ret = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                               "Filesystem::get(URI, Configuration, String)");
+                        jURI, jConfiguration, jUserString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
                 goto done;
             }
             jFS = jVal.l;
         }
     }
-
-done:
-    if (jFS) {
-        /* Create a global reference for this fs */
-        gFsRef = (*env)->NewGlobalRef(env, jFS);
+    jRet = (*env)->NewGlobalRef(env, jFS);
+    if (!jRet) {
+        ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+                    "hdfsBuilderConnect(%s)",
+                    hdfsBuilderToStr(bld, buf, sizeof(buf)));
+        goto done;
     }
+    ret = 0;
 
+done:
     // Release unnecessary local references
     destroyLocalReference(env, jConfiguration);
     destroyLocalReference(env, jFS);
@@ -679,9 +598,11 @@ done:
     free(cURI);
     free(bld);
 
-    if (ret)
+    if (ret) {
         errno = ret;
-    return (hdfsFS)gFsRef;
+        return NULL;
+    }
+    return (hdfsFS)jRet;
 }
 
 int hdfsDisconnect(hdfsFS fs)
@@ -691,6 +612,7 @@ int hdfsDisconnect(hdfsFS fs)
 
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
+    int ret;
 
     if (env == NULL) {
       errno = EINTERNAL;
@@ -700,24 +622,25 @@ int hdfsDisconnect(hdfsFS fs)
     //Parameters
     jobject jFS = (jobject)fs;
 
-    //Caught exception
-    jthrowable jExc = NULL;
-
     //Sanity check
     if (fs == NULL) {
         errno = EBADF;
         return -1;
     }
 
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "close", "()V") != 0) {
-        errno = errnoFromException(jExc, env, "Filesystem::close");
-        return -1;
+    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
+                     "close", "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsDisconnect: FileSystem#close");
+    } else {
+        ret = 0;
     }
-
-    //Release unnecessary references
     (*env)->DeleteGlobalRef(env, jFS);
-
+    if (ret) {
+        errno = ret;
+        return -1;
+    }
     return 0;
 }
 
@@ -740,7 +663,14 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
       return NULL;
     }
 
+    jstring jStrBufferSize = NULL, jStrReplication = NULL;
+    jstring jStrBlockSize = NULL;
+    jobject jConfiguration = NULL, jPath = NULL, jFile = NULL;
     jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jvalue jVal;
+    hdfsFile file = NULL;
+    int ret;
 
     if (flags & O_RDWR) {
       fprintf(stderr, "ERROR: cannot open an hdfs file in O_RDWR mode\n");
@@ -760,43 +690,52 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
       JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_OSTRM)) :
       JMETHOD2(JPARAM(HADOOP_PATH), "ZISJ", JPARAM(HADOOP_OSTRM));
 
-    /* Return value */
-    hdfsFile file = NULL;
-
     /* Create an object of org.apache.hadoop.fs.Path */
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
-        return NULL; 
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsOpenFile(%s): constructNewObjectOfPath", path);
+        goto done;
     }
 
     /* Get the Configuration object from the FileSystem object */
-    jvalue  jVal;
-    jobject jConfiguration = NULL;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "getConf", JMETHOD1("", JPARAM(HADOOP_CONF))) != 0) {
-        errno = errnoFromException(jExc, env, "get configuration object "
-                                   "from filesystem");
-        destroyLocalReference(env, jPath);
-        return NULL;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                     "getConf", JMETHOD1("", JPARAM(HADOOP_CONF)));
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsOpenFile(%s): FileSystem#getConf", path);
+        goto done;
     }
     jConfiguration = jVal.l;
 
     jint jBufferSize = bufferSize;
     jshort jReplication = replication;
     jlong jBlockSize = blockSize;
-    jstring jStrBufferSize = (*env)->NewStringUTF(env, "io.file.buffer.size"); 
-    jstring jStrReplication = (*env)->NewStringUTF(env, "dfs.replication");
-    jstring jStrBlockSize = (*env)->NewStringUTF(env, "dfs.block.size");
-
+    jStrBufferSize = (*env)->NewStringUTF(env, "io.file.buffer.size"); 
+    if (!jStrBufferSize) {
+        ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, "OOM");
+        goto done;
+    }
+    jStrReplication = (*env)->NewStringUTF(env, "dfs.replication");
+    if (!jStrReplication) {
+        ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, "OOM");
+        goto done;
+    }
+    jStrBlockSize = (*env)->NewStringUTF(env, "dfs.block.size");
+    if (!jStrBlockSize) {
+        ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, "OOM");
+        goto done;
+    }
 
-    //bufferSize
     if (!bufferSize) {
-        if (invokeMethod(env, &jVal, &jExc, INSTANCE, jConfiguration, 
+        jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration, 
                          HADOOP_CONF, "getInt", "(Ljava/lang/String;I)I",
-                         jStrBufferSize, 4096) != 0) {
-            errno = errnoFromException(jExc, env, "org.apache.hadoop.conf."
-                                       "Configuration::getInt");
+                         jStrBufferSize, 4096);
+        if (jthr) {
+            ret = printExceptionAndFree(env, jthr, NOPRINT_EXC_FILE_NOT_FOUND |
+                NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_UNRESOLVED_LINK,
+                "hdfsOpenFile(%s): Configuration#getInt(io.file.buffer.size)",
+                path);
             goto done;
         }
         jBufferSize = jVal.i;
@@ -806,11 +745,13 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
         //replication
 
         if (!replication) {
-            if (invokeMethod(env, &jVal, &jExc, INSTANCE, jConfiguration, 
+            jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration, 
                              HADOOP_CONF, "getInt", "(Ljava/lang/String;I)I",
-                             jStrReplication, 1) != 0) {
-                errno = errnoFromException(jExc, env, "org.apache.hadoop.conf."
-                                           "Configuration::getInt");
+                             jStrReplication, 1);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsOpenFile(%s): Configuration#getInt(dfs.replication)",
+                    path);
                 goto done;
             }
             jReplication = jVal.i;
@@ -818,12 +759,13 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
         
         //blockSize
         if (!blockSize) {
-            if (invokeMethod(env, &jVal, &jExc, INSTANCE, jConfiguration, 
+            jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration, 
                              HADOOP_CONF, "getLong", "(Ljava/lang/String;J)J",
-                             jStrBlockSize, (jlong)67108864)) {
-                errno = errnoFromException(jExc, env, "org.apache.hadoop.conf."
-                                           "FileSystem::%s(%s)", method,
-                                           signature);
+                             jStrBlockSize, (jlong)67108864);
+            if (jthr) {
+                ret  = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                    "hdfsOpenFile(%s): Configuration#getLong(dfs.block.size)",
+                    path);
                 goto done;
             }
             jBlockSize = jVal.j;
@@ -835,82 +777,79 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
 
     // READ?
     if ((flags & O_WRONLY) == 0) {
-      if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                       method, signature, jPath, jBufferSize)) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.conf."
-                                   "FileSystem::%s(%s)", method,
-                                   signature);
-        goto done;
-      }
+        jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                       method, signature, jPath, jBufferSize);
     }  else if ((flags & O_WRONLY) && (flags & O_APPEND)) {
-      // WRITE/APPEND?
-       if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                       method, signature, jPath)) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.conf."
-                                   "FileSystem::%s(%s)", method,
-                                   signature);
-        goto done;
-      }
+        // WRITE/APPEND?
+       jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                       method, signature, jPath);
     } else {
         // WRITE/CREATE
         jboolean jOverWrite = 1;
-        if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
+        jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                          method, signature, jPath, jOverWrite,
-                         jBufferSize, jReplication, jBlockSize)) {
-            errno = errnoFromException(jExc, env, "org.apache.hadoop.conf."
-                                       "FileSystem::%s(%s)", method,
-                                       signature);
-            goto done;
-        }
+                         jBufferSize, jReplication, jBlockSize);
     }
-  
-    file = malloc(sizeof(struct hdfsFile_internal));
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsOpenFile(%s): FileSystem#%s(%s)", path, method, signature);
+        goto done;
+    }
+    jFile = jVal.l;
+
+    file = calloc(1, sizeof(struct hdfsFile_internal));
     if (!file) {
-        errno = ENOMEM;
-    } else {
-        file->file = (*env)->NewGlobalRef(env, jVal.l);
-        file->type = (((flags & O_WRONLY) == 0) ? INPUT : OUTPUT);
-        file->flags = 0;
-
-        destroyLocalReference(env, jVal.l);
-
-        if ((flags & O_WRONLY) == 0) {
-          // Try a test read to see if we can do direct reads
-          errno = 0;
-          char buf;
-          if (readDirect(fs, file, &buf, 0) == 0) {
+        fprintf(stderr, "hdfsOpenFile(%s): OOM create hdfsFile\n", path);
+        ret = ENOMEM;
+        goto done;
+    }
+    file->file = (*env)->NewGlobalRef(env, jFile);
+    if (!file->file) {
+        ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsOpenFile(%s): NewGlobalRef", path); 
+        goto done;
+    }
+    file->type = (((flags & O_WRONLY) == 0) ? INPUT : OUTPUT);
+    file->flags = 0;
+
+    if ((flags & O_WRONLY) == 0) {
+        // Try a test read to see if we can do direct reads
+        char buf;
+        if (readDirect(fs, file, &buf, 0) == 0) {
             // Success - 0-byte read should return 0
             file->flags |= HDFS_FILE_SUPPORTS_DIRECT_READ;
-          } else {
-            if (errno != ENOTSUP) {
-              // Unexpected error. Clear it, don't set the direct flag.
-              fprintf(stderr,
-                      "WARN: Unexpected error %d when testing "
-                      "for direct read compatibility\n", errno);
-              errno = 0;
-              goto done;
-            }
-          }
-          errno = 0;
+        } else if (errno != ENOTSUP) {
+            // Unexpected error. Clear it, don't set the direct flag.
+            fprintf(stderr,
+                  "hdfsOpenFile(%s): WARN: Unexpected error %d when testing "
+                  "for direct read compatibility\n", path, errno);
         }
     }
+    ret = 0;
 
-    done:
-
-    //Delete unnecessary local references
+done:
     destroyLocalReference(env, jStrBufferSize);
     destroyLocalReference(env, jStrReplication);
     destroyLocalReference(env, jStrBlockSize);
     destroyLocalReference(env, jConfiguration); 
     destroyLocalReference(env, jPath); 
-
+    destroyLocalReference(env, jFile); 
+    if (ret) {
+        if (file) {
+            if (file->file) {
+                (*env)->DeleteGlobalRef(env, file->file);
+            }
+            free(file);
+        }
+        errno = ret;
+        return NULL;
+    }
     return file;
 }
 
-
-
 int hdfsCloseFile(hdfsFS fs, hdfsFile file)
 {
+    int ret;
     // JAVA EQUIVALENT:
     //  file.close 
 
@@ -918,15 +857,12 @@ int hdfsCloseFile(hdfsFS fs, hdfsFile fi
     JNIEnv* env = getJNIEnv();
 
     if (env == NULL) {
-      errno = EINTERNAL;
-      return -1;
+        errno = EINTERNAL;
+        return -1;
     }
 
-    //Parameters
-    jobject jStream = (jobject)(file ? file->file : NULL);
-
     //Caught exception
-    jthrowable jExc = NULL;
+    jthrowable jthr;
 
     //Sanity check
     if (!file || file->type == UNINITIALIZED) {
@@ -938,49 +874,66 @@ int hdfsCloseFile(hdfsFS fs, hdfsFile fi
     const char* interface = (file->type == INPUT) ? 
         HADOOP_ISTRM : HADOOP_OSTRM;
   
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jStream, interface,
-                     "close", "()V") != 0) {
-        errno = errnoFromException(jExc, env, "%s::close", interface);
-        return -1;
+    jthr = invokeMethod(env, NULL, INSTANCE, file->file, interface,
+                     "close", "()V");
+    if (jthr) {
+        const char *interfaceShortName = (file->type == INPUT) ? 
+            "FSDataInputStream" : "FSDataOutputStream";
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                "%s#close", interfaceShortName);
+    } else {
+        ret = 0;
     }
 
     //De-allocate memory
+    (*env)->DeleteGlobalRef(env, file->file);
     free(file);
-    (*env)->DeleteGlobalRef(env, jStream);
 
+    if (ret) {
+        errno = ret;
+        return -1;
+    }
     return 0;
 }
 
-
-
 int hdfsExists(hdfsFS fs, const char *path)
 {
     JNIEnv *env = getJNIEnv();
     if (env == NULL) {
-      errno = EINTERNAL;
-      return -1;
+        errno = EINTERNAL;
+        return -1;
     }
 
-    jobject jPath = constructNewObjectOfPath(env, path);
+    jobject jPath;
     jvalue  jVal;
-    jthrowable jExc = NULL;
     jobject jFS = (jobject)fs;
-
-    if (jPath == NULL) {
+    jthrowable jthr;
+    
+    if (path == NULL) {
+        errno = EINVAL;
         return -1;
     }
-
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "exists", JMETHOD1(JPARAM(HADOOP_PATH), "Z"),
-                     jPath) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::exists");
-        destroyLocalReference(env, jPath);
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsExists: constructNewObjectOfPath");
         return -1;
     }
-
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+            "exists", JMETHOD1(JPARAM(HADOOP_PATH), "Z"), jPath);
     destroyLocalReference(env, jPath);
-    return jVal.z ? 0 : -1;
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsExists: invokeMethod(%s)",
+            JMETHOD1(JPARAM(HADOOP_PATH), "Z"));
+        return -1;
+    }
+    if (jVal.z) {
+        return 0;
+    } else {
+        errno = ENOENT;
+        return -1;
+    }
 }
 
 // Checks input file for readiness for reading.
@@ -1005,35 +958,14 @@ static int readPrepare(JNIEnv* env, hdfs
     return 0;
 }
 
-// Common error-handling code between read paths.
-static int handleReadResult(int success, jvalue jVal, jthrowable jExc,
-                            JNIEnv* env)
-{
-  int noReadBytes;
-  if (success != 0) {
-    if ((*env)->ExceptionCheck(env)) {
-      errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                 "FSDataInputStream::read");
-    }
-    noReadBytes = -1;
-  } else {
-    noReadBytes = jVal.i;
-    if (noReadBytes == 0) {
-      // 0 from Java means try again, which is EINTR here
-      errno = EINTR;
-      noReadBytes = -1;
-    } else if (noReadBytes < 0) {
-      // -1 from Java is EOF, which is 0 here
-      errno = 0;
-      noReadBytes = 0;
-    }
-  }
-
-  return noReadBytes;
-}
-
 tSize hdfsRead(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
 {
+    if (length == 0) {
+        return 0;
+    } else if (length < 0) {
+        errno = EINVAL;
+        return -1;
+    }
     if (f->flags & HDFS_FILE_SUPPORTS_DIRECT_READ) {
       return readDirect(fs, f, buffer, length);
     }
@@ -1056,27 +988,45 @@ tSize hdfsRead(hdfsFS fs, hdfsFile f, vo
     }
 
     jbyteArray jbRarray;
-    jint noReadBytes = 0;
+    jint noReadBytes = length;
     jvalue jVal;
-    jthrowable jExc = NULL;
+    jthrowable jthr;
 
     //Read the requisite bytes
     jbRarray = (*env)->NewByteArray(env, length);
+    if (!jbRarray) {
+        errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsRead: NewByteArray");
+        return -1;
+    }
 
-    int success = invokeMethod(env, &jVal, &jExc, INSTANCE, jInputStream, HADOOP_ISTRM,
+    jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream, HADOOP_ISTRM,
                                "read", "([B)I", jbRarray);
-
-    noReadBytes = handleReadResult(success, jVal, jExc, env);;
-
-    if (noReadBytes > 0) {
-      (*env)->GetByteArrayRegion(env, jbRarray, 0, noReadBytes, buffer);
+    if (jthr) {
+        destroyLocalReference(env, jbRarray);
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsRead: FSDataInputStream#read");
+        return -1;
+    }
+    if (jVal.i < 0) {
+        // EOF
+        destroyLocalReference(env, jbRarray);
+        return 0;
+    } else if (jVal.i == 0) {
+        destroyLocalReference(env, jbRarray);
+        errno = EINTR;
+        return -1;
     }
-
+    (*env)->GetByteArrayRegion(env, jbRarray, 0, noReadBytes, buffer);
     destroyLocalReference(env, jbRarray);
-
-    return noReadBytes;
-}
-
+    if ((*env)->ExceptionCheck(env)) {
+        errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsRead: GetByteArrayRegion");
+        return -1;
+    }
+    return jVal.i;
+}
+
 // Reads using the read(ByteBuffer) API, which does fewer copies
 tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
 {
@@ -1096,63 +1046,53 @@ tSize readDirect(hdfsFS fs, hdfsFile f, 
       return -1;
     }
 
-    jint noReadBytes = 0;
     jvalue jVal;
-    jthrowable jExc = NULL;
+    jthrowable jthr;
 
     //Read the requisite bytes
     jobject bb = (*env)->NewDirectByteBuffer(env, buffer, length);
     if (bb == NULL) {
-      fprintf(stderr, "Could not allocate ByteBuffer");
-      if ((*env)->ExceptionCheck(env)) {
-        errno = errnoFromException(NULL, env, "JNIEnv::NewDirectByteBuffer");
-      } else {
-        errno = ENOMEM; // Best guess if there's no exception waiting
-      }
-      return -1;
+        errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "readDirect: NewDirectByteBuffer");
+        return -1;
     }
 
-    int success = invokeMethod(env, &jVal, &jExc, INSTANCE, jInputStream,
-                               HADOOP_ISTRM, "read", "(Ljava/nio/ByteBuffer;)I",
-                               bb);
-
-    noReadBytes = handleReadResult(success, jVal, jExc, env);
-
+    jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
+        HADOOP_ISTRM, "read", "(Ljava/nio/ByteBuffer;)I", bb);
     destroyLocalReference(env, bb);
-
-    return noReadBytes;
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "readDirect: FSDataInputStream#read");
+        return -1;
+    }
+    return (jVal.i < 0) ? 0 : jVal.i;
 }
 
-
-  
 tSize hdfsPread(hdfsFS fs, hdfsFile f, tOffset position,
                 void* buffer, tSize length)
 {
-    // JAVA EQUIVALENT:
-    //  byte [] bR = new byte[length];
-    //  fis.read(pos, bR, 0, length);
-
-    //Get the JNIEnv* corresponding to current thread
-    JNIEnv* env = getJNIEnv();
-    if (env == NULL) {
-      errno = EINTERNAL;
-      return -1;
-    }
-
-    //Parameters
-    jobject jInputStream = (jobject)(f ? f->file : NULL);
-
+    JNIEnv* env;
     jbyteArray jbRarray;
-    jint noReadBytes = 0;
     jvalue jVal;
-    jthrowable jExc = NULL;
+    jthrowable jthr;
 
-    //Sanity check
+    if (length == 0) {
+        return 0;
+    } else if (length < 0) {
+        errno = EINVAL;
+        return -1;
+    }
     if (!f || f->type == UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
 
+    env = getJNIEnv();
+    if (env == NULL) {
+      errno = EINTERNAL;
+      return -1;
+    }
+
     //Error checking... make sure that this file is 'readable'
     if (f->type != INPUT) {
         fprintf(stderr, "Cannot read from a non-InputStream object!\n");
@@ -1160,34 +1100,42 @@ tSize hdfsPread(hdfsFS fs, hdfsFile f, t
         return -1;
     }
 
-    //Read the requisite bytes
+    // JAVA EQUIVALENT:
+    //  byte [] bR = new byte[length];
+    //  fis.read(pos, bR, 0, length);
     jbRarray = (*env)->NewByteArray(env, length);
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jInputStream, HADOOP_ISTRM,
-                     "read", "(J[BII)I", position, jbRarray, 0, length) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FSDataInputStream::read");
-        noReadBytes = -1;
-    }
-    else {
-        noReadBytes = jVal.i;
-        if (noReadBytes > 0) {
-            (*env)->GetByteArrayRegion(env, jbRarray, 0, noReadBytes, buffer);
-        }  else {
-            //This is a valid case: there aren't any bytes left to read!
-          if (noReadBytes == 0 || noReadBytes < -1) {
-            fprintf(stderr, "WARN: FSDataInputStream.read returned invalid return code - libhdfs returning EOF, i.e., 0: %d\n", noReadBytes);
-          }
-            noReadBytes = 0;
-        }
-        errno = 0;
+    if (!jbRarray) {
+        errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsPread: NewByteArray");
+        return -1;
     }
+    jthr = invokeMethod(env, &jVal, INSTANCE, f->file, HADOOP_ISTRM,
+                     "read", "(J[BII)I", position, jbRarray, 0, length);
+    if (jthr) {
+        destroyLocalReference(env, jbRarray);
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsPread: FSDataInputStream#read");
+        return -1;
+    }
+    if (jVal.i < 0) {
+        // EOF
+        destroyLocalReference(env, jbRarray);
+        return 0;
+    } else if (jVal.i == 0) {
+        destroyLocalReference(env, jbRarray);
+        errno = EINTR;
+        return -1;
+    }
+    (*env)->GetByteArrayRegion(env, jbRarray, 0, jVal.i, buffer);
     destroyLocalReference(env, jbRarray);
-
-    return noReadBytes;
+    if ((*env)->ExceptionCheck(env)) {
+        errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsPread: GetByteArrayRegion");
+        return -1;
+    }
+    return jVal.i;
 }
 
-
-
 tSize hdfsWrite(hdfsFS fs, hdfsFile f, const void* buffer, tSize length)
 {
     // JAVA EQUIVALENT
@@ -1201,18 +1149,15 @@ tSize hdfsWrite(hdfsFS fs, hdfsFile f, c
       return -1;
     }
 
-    //Parameters
-    jobject jOutputStream = (jobject)(f ? f->file : 0);
-    jbyteArray jbWarray;
-
-    //Caught exception
-    jthrowable jExc = NULL;
-
     //Sanity check
     if (!f || f->type == UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
+
+    jobject jOutputStream = f->file;
+    jbyteArray jbWarray;
+    jthrowable jthr;
     
     if (length < 0) {
     	errno = EINVAL;
@@ -1226,28 +1171,40 @@ tSize hdfsWrite(hdfsFS fs, hdfsFile f, c
         return -1;
     }
 
-    // 'length' equals 'zero' is a valid use-case according to Posix!
-    if (length != 0) {
-        //Write the requisite bytes into the file
-        jbWarray = (*env)->NewByteArray(env, length);
-        (*env)->SetByteArrayRegion(env, jbWarray, 0, length, buffer);
-        if (invokeMethod(env, NULL, &jExc, INSTANCE, jOutputStream,
-                         HADOOP_OSTRM, "write",
-                         "([B)V", jbWarray) != 0) {
-            errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                       "FSDataOutputStream::write");
-            length = -1;
-        }
+    if (length < 0) {
+        errno = EINVAL;
+        return -1;
+    }
+    if (length == 0) {
+        return 0;
+    }
+    //Write the requisite bytes into the file
+    jbWarray = (*env)->NewByteArray(env, length);
+    if (!jbWarray) {
+        errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsWrite: NewByteArray");
+        return -1;
+    }
+    (*env)->SetByteArrayRegion(env, jbWarray, 0, length, buffer);
+    if ((*env)->ExceptionCheck(env)) {
         destroyLocalReference(env, jbWarray);
+        errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsWrite(length = %d): SetByteArrayRegion", length);
+        return -1;
     }
-
-    //Return no. of bytes succesfully written (libc way)
-    //i.e. 'length' itself! ;-)
+    jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
+            HADOOP_OSTRM, "write", "([B)V", jbWarray);
+    destroyLocalReference(env, jbWarray);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsWrite: FSDataOutputStream#write");
+        return -1;
+    }
+    // Unlike most Java streams, FSDataOutputStream never does partial writes.
+    // If we succeeded, all the data was written.
     return length;
 }
 
-
-
 int hdfsSeek(hdfsFS fs, hdfsFile f, tOffset desiredPos) 
 {
     // JAVA EQUIVALENT
@@ -1260,25 +1217,21 @@ int hdfsSeek(hdfsFS fs, hdfsFile f, tOff
       return -1;
     }
 
-    //Parameters
-    jobject jInputStream = (jobject)(f ? f->file : 0);
-
-    //Caught exception
-    jthrowable jExc = NULL;
-
     //Sanity check
     if (!f || f->type != INPUT) {
         errno = EBADF;
         return -1;
     }
 
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jInputStream, HADOOP_ISTRM,
-                     "seek", "(J)V", desiredPos) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FSDataInputStream::seek");
+    jobject jInputStream = f->file;
+    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jInputStream,
+            HADOOP_ISTRM, "seek", "(J)V", desiredPos);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsSeek(desiredPos=%" PRId64 ")"
+            ": FSDataInputStream#seek", desiredPos);
         return -1;
     }
-
     return 0;
 }
 
@@ -1296,34 +1249,29 @@ tOffset hdfsTell(hdfsFS fs, hdfsFile f)
       return -1;
     }
 
-    //Parameters
-    jobject jStream = (jobject)(f ? f->file : 0);
-
     //Sanity check
     if (!f || f->type == UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
 
+    //Parameters
+    jobject jStream = f->file;
     const char* interface = (f->type == INPUT) ?
         HADOOP_ISTRM : HADOOP_OSTRM;
-
-    jlong currentPos  = -1;
     jvalue jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStream,
-                     interface, "getPos", "()J") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FSDataInputStream::getPos");
+    jthrowable jthr = invokeMethod(env, &jVal, INSTANCE, jStream,
+                     interface, "getPos", "()J");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsTell: %s#getPos",
+            ((f->type == INPUT) ? "FSDataInputStream" :
+                                 "FSDataOutputStream"));
         return -1;
     }
-    currentPos = jVal.j;
-
-    return (tOffset)currentPos;
+    return jVal.j;
 }
 
-
-
 int hdfsFlush(hdfsFS fs, hdfsFile f) 
 {
     // JAVA EQUIVALENT
@@ -1336,30 +1284,21 @@ int hdfsFlush(hdfsFS fs, hdfsFile f) 
       return -1;
     }
 
-    //Parameters
-    jobject jOutputStream = (jobject)(f ? f->file : 0);
-
-    //Caught exception
-    jthrowable jExc = NULL;
-
     //Sanity check
     if (!f || f->type != OUTPUT) {
         errno = EBADF;
         return -1;
     }
-
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jOutputStream, 
-                     HADOOP_OSTRM, "flush", "()V") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FSDataInputStream::flush");
+    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, f->file, 
+                     HADOOP_OSTRM, "flush", "()V");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsFlush: FSDataInputStream#flush");
         return -1;
     }
-
     return 0;
 }
 
-
-
 int hdfsHFlush(hdfsFS fs, hdfsFile f)
 {
     //Get the JNIEnv* corresponding to current thread
@@ -1369,29 +1308,23 @@ int hdfsHFlush(hdfsFS fs, hdfsFile f)
       return -1;
     }
 
-    //Parameters
-    jobject jOutputStream = (jobject)(f ? f->file : 0);
-
-    //Caught exception
-    jthrowable jExc = NULL;
-
     //Sanity check
     if (!f || f->type != OUTPUT) {
         errno = EBADF;
         return -1;
     }
 
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jOutputStream,
-                     HADOOP_OSTRM, "hflush", "()V") != 0) {
-        errno = errnoFromException(jExc, env, HADOOP_OSTRM "::hflush");
+    jobject jOutputStream = f->file;
+    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
+                     HADOOP_OSTRM, "hflush", "()V");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsHFlush: FSDataOutputStream#hflush");
         return -1;
     }
-
     return 0;
 }
 
-
-
 int hdfsAvailable(hdfsFS fs, hdfsFile f)
 {
     // JAVA EQUIVALENT
@@ -1404,37 +1337,30 @@ int hdfsAvailable(hdfsFS fs, hdfsFile f)
       return -1;
     }
 
-    //Parameters
-    jobject jInputStream = (jobject)(f ? f->file : 0);
-
-    //Caught exception
-    jthrowable jExc = NULL;
-
     //Sanity check
     if (!f || f->type != INPUT) {
         errno = EBADF;
         return -1;
     }
 
-    jint available = -1;
+    //Parameters
+    jobject jInputStream = f->file;
     jvalue jVal;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jInputStream, 
-                     HADOOP_ISTRM, "available", "()I") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FSDataInputStream::available");
+    jthrowable jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
+                     HADOOP_ISTRM, "available", "()I");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsAvailable: FSDataInputStream#available");
         return -1;
     }
-    available = jVal.i;
-
-    return available;
+    return jVal.i;
 }
 
-
-
-int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
+static int hdfsCopyImpl(hdfsFS srcFS, const char* src, hdfsFS dstFS,
+        const char* dst, jboolean deleteSource)
 {
     //JAVA EQUIVALENT
-    //  FileUtil::copy(srcFS, srcPath, dstFS, dstPath,
+    //  FileUtil#copy(srcFS, srcPath, dstFS, dstPath,
     //                 deleteSource = false, conf)
 
     //Get the JNIEnv* corresponding to current thread
@@ -1447,139 +1373,80 @@ int hdfsCopy(hdfsFS srcFS, const char* s
     //Parameters
     jobject jSrcFS = (jobject)srcFS;
     jobject jDstFS = (jobject)dstFS;
-    jobject jSrcPath = NULL;
-    jobject jDstPath = NULL;
+    jobject jConfiguration = NULL, jSrcPath = NULL, jDstPath = NULL;
+    jthrowable jthr;
+    jvalue jVal;
+    int ret;
 
-    jSrcPath = constructNewObjectOfPath(env, src);
-    if (jSrcPath == NULL) {
-        return -1;
+    jthr = constructNewObjectOfPath(env, src, &jSrcPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsCopyImpl(src=%s): constructNewObjectOfPath", src);
+        goto done;
     }
-
-    jDstPath = constructNewObjectOfPath(env, dst);
-    if (jDstPath == NULL) {
-        destroyLocalReference(env, jSrcPath);
-        return -1;
+    jthr = constructNewObjectOfPath(env, dst, &jDstPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsCopyImpl(dst=%s): constructNewObjectOfPath", dst);
+        goto done;
     }
 
-    int retval = 0;
-
     //Create the org.apache.hadoop.conf.Configuration object
-    jobject jConfiguration =
-        constructNewObjectOfClass(env, NULL, HADOOP_CONF, "()V");
-    if (jConfiguration == NULL) {
-        fprintf(stderr, "Can't construct instance of class "
-                "org.apache.hadoop.conf.Configuration\n");
-        destroyLocalReference(env, jSrcPath);
-        destroyLocalReference(env, jDstPath);
-        errno = EINTERNAL;
-        return -1;
+    jthr = constructNewObjectOfClass(env, &jConfiguration,
+                                     HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsCopyImpl: Configuration constructor");
+        goto done;
     }
 
-    //FileUtil::copy
-    jboolean deleteSource = 0; //Only copy
-    jvalue jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, STATIC, 
-                     NULL, "org/apache/hadoop/fs/FileUtil", "copy",
-                     "(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;ZLorg/apache/hadoop/conf/Configuration;)Z",
-                     jSrcFS, jSrcPath, jDstFS, jDstPath, deleteSource, 
-                     jConfiguration) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileUtil::copy");
-        retval = -1;
+    //FileUtil#copy
+    jthr = invokeMethod(env, &jVal, STATIC,
+            NULL, "org/apache/hadoop/fs/FileUtil", "copy",
+            "(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
+            "Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
+            "ZLorg/apache/hadoop/conf/Configuration;)Z",
+            jSrcFS, jSrcPath, jDstFS, jDstPath, deleteSource, 
+            jConfiguration);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsCopyImpl(src=%s, dst=%s, deleteSource=%d): "
+            "FileUtil#copy", src, dst, deleteSource);
         goto done;
     }
+    if (!jVal.z) {
+        ret = EIO;
+        goto done;
+    }
+    ret = 0;
 
-    done:
-
-    //Delete unnecessary local references
+done:
     destroyLocalReference(env, jConfiguration);
     destroyLocalReference(env, jSrcPath);
     destroyLocalReference(env, jDstPath);
   
-    return retval;
+    if (ret) {
+        errno = ret;
+        return -1;
+    }
+    return 0;
 }
 
-
+int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
+{
+    return hdfsCopyImpl(srcFS, src, dstFS, dst, 0);
+}
 
 int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
 {
-    //JAVA EQUIVALENT
-    //  FileUtil::copy(srcFS, srcPath, dstFS, dstPath,
-    //                 deleteSource = true, conf)
-
-    //Get the JNIEnv* corresponding to current thread
-    JNIEnv* env = getJNIEnv();
-    if (env == NULL) {
-      errno = EINTERNAL;
-      return -1;
-    }
-
-
-    //Parameters
-    jobject jSrcFS = (jobject)srcFS;
-    jobject jDstFS = (jobject)dstFS;
-
-    jobject jSrcPath = NULL;
-    jobject jDstPath = NULL;
-
-    jSrcPath = constructNewObjectOfPath(env, src);
-    if (jSrcPath == NULL) {
-        return -1;
-    }
-
-    jDstPath = constructNewObjectOfPath(env, dst);
-    if (jDstPath == NULL) {
-        destroyLocalReference(env, jSrcPath);
-        return -1;
-    }
-
-    int retval = 0;
-
-    //Create the org.apache.hadoop.conf.Configuration object
-    jobject jConfiguration =
-        constructNewObjectOfClass(env, NULL, HADOOP_CONF, "()V");
-    if (jConfiguration == NULL) {
-        fprintf(stderr, "Can't construct instance of class "
-                "org.apache.hadoop.conf.Configuration\n");
-        destroyLocalReference(env, jSrcPath);
-        destroyLocalReference(env, jDstPath);
-        errno = EINTERNAL;
-        return -1;
-    }
-
-    //FileUtil::copy
-    jboolean deleteSource = 1; //Delete src after copy
-    jvalue jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, STATIC, NULL,
-                     "org/apache/hadoop/fs/FileUtil", "copy",
-                "(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;ZLorg/apache/hadoop/conf/Configuration;)Z",
-                     jSrcFS, jSrcPath, jDstFS, jDstPath, deleteSource, 
-                     jConfiguration) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileUtil::copy(move)");
-        retval = -1;
-        goto done;
-    }
-
-    done:
-
-    //Delete unnecessary local references
-    destroyLocalReference(env, jConfiguration);
-    destroyLocalReference(env, jSrcPath);
-    destroyLocalReference(env, jDstPath);
-  
-    return retval;
+    return hdfsCopyImpl(srcFS, src, dstFS, dst, 1);
 }
 
-
-
 int hdfsDelete(hdfsFS fs, const char* path, int recursive)
 {
     // JAVA EQUIVALENT:
-    //  File f = new File(path);
-    //  bool retval = fs.delete(f);
+    //  Path p = new Path(path);
+    //  bool retval = fs.delete(p, recursive);
 
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
@@ -1589,30 +1456,32 @@ int hdfsDelete(hdfsFS fs, const char* pa
     }
 
     jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jobject jPath;
+    jvalue jVal;
 
-    //Create an object of java.io.File
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsDelete(path=%s): constructNewObjectOfPath", path);
         return -1;
     }
-
-    //Delete the file
-    jvalue jVal;
-    jthrowable jExc = NULL;
     jboolean jRecursive = recursive ? JNI_TRUE : JNI_FALSE;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "delete", "(Lorg/apache/hadoop/fs/Path;Z)Z",
-                     jPath, jRecursive) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::delete");
-        destroyLocalReference(env, jPath);
+                     jPath, jRecursive);
+    destroyLocalReference(env, jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsDelete(path=%s, recursive=%d): "
+            "FileSystem#delete", path, recursive);
         return -1;
     }
-
-    //Delete unnecessary local references
-    destroyLocalReference(env, jPath);
-
-    return (jVal.z) ? 0 : -1;
+    if (!jVal.z) {
+        errno = EIO;
+        return -1;
+    }
+    return 0;
 }
 
 
@@ -1632,40 +1501,45 @@ int hdfsRename(hdfsFS fs, const char* ol
     }
 
     jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jobject jOldPath = NULL, jNewPath = NULL;
+    int ret = -1;
+    jvalue jVal;
 
-    //Create objects of org.apache.hadoop.fs.Path
-    jobject jOldPath = NULL;
-    jobject jNewPath = NULL;
-
-    jOldPath = constructNewObjectOfPath(env, oldPath);
-    if (jOldPath == NULL) {
-        return -1;
+    jthr = constructNewObjectOfPath(env, oldPath, &jOldPath );
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsRename: constructNewObjectOfPath(%s)", oldPath);
+        goto done;
     }
-
-    jNewPath = constructNewObjectOfPath(env, newPath);
-    if (jNewPath == NULL) {
-        destroyLocalReference(env, jOldPath);
-        return -1;
+    jthr = constructNewObjectOfPath(env, newPath, &jNewPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsRename: constructNewObjectOfPath(%s)", newPath);
+        goto done;
     }
 
-    //Rename the file
-    jvalue jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS, "rename",
+    // Rename the file
+    // TODO: use rename2 here?  (See HDFS-3592)
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS, "rename",
                      JMETHOD2(JPARAM(HADOOP_PATH), JPARAM(HADOOP_PATH), "Z"),
-                     jOldPath, jNewPath) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::rename");
-        destroyLocalReference(env, jOldPath);
-        destroyLocalReference(env, jNewPath);
-        return -1;
+                     jOldPath, jNewPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsRename(oldPath=%s, newPath=%s): FileSystem#rename",
+            oldPath, newPath);
+        goto done;
+    }
+    if (!jVal.z) {
+        errno = EIO;
+        goto done;
     }
+    ret = 0;
 
-    //Delete unnecessary local references
+done:
     destroyLocalReference(env, jOldPath);
     destroyLocalReference(env, jNewPath);
-
-    return (jVal.z) ? 0 : -1;
+    return ret;
 }
 
 
@@ -1683,45 +1557,67 @@ char* hdfsGetWorkingDirectory(hdfsFS fs,
       return NULL;
     }
 
-    jobject jFS = (jobject)fs;
     jobject jPath = NULL;
+    jstring jPathString = NULL;
+    jobject jFS = (jobject)fs;
     jvalue jVal;
-    jthrowable jExc = NULL;
+    jthrowable jthr;
+    int ret;
+    const char *jPathChars = NULL;
 
-    //FileSystem::getWorkingDirectory()
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS,
+    //FileSystem#getWorkingDirectory()
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
                      HADOOP_FS, "getWorkingDirectory",
-                     "()Lorg/apache/hadoop/fs/Path;") != 0 ||
-        jVal.l == NULL) {
-        errno = errnoFromException(jExc, env, "FileSystem::"
-                                   "getWorkingDirectory");
-        return NULL;
+                     "()Lorg/apache/hadoop/fs/Path;");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsGetWorkingDirectory: FileSystem#getWorkingDirectory");
+        goto done;
     }
     jPath = jVal.l;
+    if (!jPath) {
+        fprintf(stderr, "hdfsGetWorkingDirectory: "
+            "FileSystem#getWorkingDirectory returned NULL");
+        ret = -EIO;
+        goto done;
+    }
 
-    //Path::toString()
-    jstring jPathString;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jPath, 
+    //Path#toString()
+    jthr = invokeMethod(env, &jVal, INSTANCE, jPath, 
                      "org/apache/hadoop/fs/Path", "toString",
-                     "()Ljava/lang/String;") != 0) { 
-        errno = errnoFromException(jExc, env, "Path::toString");
-        destroyLocalReference(env, jPath);
-        return NULL;
+                     "()Ljava/lang/String;");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsGetWorkingDirectory: Path#toString");
+        goto done;
     }
     jPathString = jVal.l;
-
-    const char *jPathChars = (const char*)
-        ((*env)->GetStringUTFChars(env, jPathString, NULL));
+    jPathChars = (*env)->GetStringUTFChars(env, jPathString, NULL);
+    if (!jPathChars) {
+        ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+            "hdfsGetWorkingDirectory: GetStringUTFChars");
+        goto done;
+    }
 
     //Copy to user-provided buffer
-    strncpy(buffer, jPathChars, bufferSize);
-
-    //Delete unnecessary local references
-    (*env)->ReleaseStringUTFChars(env, jPathString, jPathChars);
+    ret = snprintf(buffer, bufferSize, "%s", jPathChars);
+    if (ret >= bufferSize) {
+        ret = ENAMETOOLONG;
+        goto done;
+    }
+    ret = 0;
 
-    destroyLocalReference(env, jPathString);
+done:
+    if (jPathChars) {
+        (*env)->ReleaseStringUTFChars(env, jPathString, jPathChars);
+    }
     destroyLocalReference(env, jPath);
+    destroyLocalReference(env, jPathString);
 
+    if (ret) {
+        errno = ret;
+        return NULL;
+    }
     return buffer;
 }
 
@@ -1740,28 +1636,30 @@ int hdfsSetWorkingDirectory(hdfsFS fs, c
     }
 
     jobject jFS = (jobject)fs;
-    int retval = 0;
-    jthrowable jExc = NULL;
+    jthrowable jthr;
+    jobject jPath;
 
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsSetWorkingDirectory(%s): constructNewObjectOfPath",
+            path);
         return -1;
     }
 
-    //FileSystem::setWorkingDirectory()
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jFS, HADOOP_FS,
+    //FileSystem#setWorkingDirectory()
+    jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
                      "setWorkingDirectory", 
-                     "(Lorg/apache/hadoop/fs/Path;)V", jPath) != 0) {
-        errno = errnoFromException(jExc, env, "FileSystem::"
-                                   "setWorkingDirectory");
-        retval = -1;
-    }
-
-    //Delete unnecessary local references
+                     "(Lorg/apache/hadoop/fs/Path;)V", jPath);
     destroyLocalReference(env, jPath);
-
-    return retval;
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, NOPRINT_EXC_ILLEGAL_ARGUMENT,
+            "hdfsSetWorkingDirectory(%s): FileSystem#setWorkingDirectory",
+            path);
+        return -1;
+    }
+    return 0;
 }
 
 
@@ -1779,31 +1677,41 @@ int hdfsCreateDirectory(hdfsFS fs, const
     }
 
     jobject jFS = (jobject)fs;
+    jobject jPath;
+    jthrowable jthr;
 
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsCreateDirectory(%s): constructNewObjectOfPath", path);
         return -1;
     }
 
     //Create the directory
     jvalue jVal;
     jVal.z = 0;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "mkdirs", "(Lorg/apache/hadoop/fs/Path;)Z",
-                     jPath) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::mkdirs");
-        goto done;
-    }
-
- done:
-
-    //Delete unnecessary local references
+                     jPath);
     destroyLocalReference(env, jPath);
-
-    return (jVal.z) ? 0 : -1;
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr,
+            NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
+            NOPRINT_EXC_UNRESOLVED_LINK | NOPRINT_EXC_PARENT_NOT_DIRECTORY,
+            "hdfsCreateDirectory(%s): FileSystem#mkdirs", path);
+        return -1;
+    }
+    if (!jVal.z) {
+        // It's unclear under exactly which conditions FileSystem#mkdirs
+        // is supposed to return false (as opposed to throwing an exception.)
+        // It seems like the current code never actually returns false.
+        // So we're going to translate this to EIO, since there seems to be
+        // nothing more specific we can do with it.
+        errno = EIO;
+        return -1;
+    }
+    return 0;
 }
 
 
@@ -1820,30 +1728,37 @@ int hdfsSetReplication(hdfsFS fs, const 
     }
 
     jobject jFS = (jobject)fs;
+    jthrowable jthr;
 
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
+    jobject jPath;
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsSetReplication(path=%s): constructNewObjectOfPath", path);
         return -1;
     }
 
     //Create the directory
     jvalue jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "setReplication", "(Lorg/apache/hadoop/fs/Path;S)Z",
-                     jPath, replication) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::setReplication");
-        goto done;
-    }
-
- done:
-
-    //Delete unnecessary local references
+                     jPath, replication);
     destroyLocalReference(env, jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsSetReplication(path=%s, replication=%d): "
+            "FileSystem#setReplication", path, replication);
+        return -1;
+    }
+    if (!jVal.z) {
+        // setReplication returns false "if file does not exist or is a
+        // directory."  So the nearest translation to that is ENOENT.
+        errno = ENOENT;
+        return -1;
+    }
 
-    return (jVal.z) ? 0 : -1;
+    return 0;
 }
 
 int hdfsChown(hdfsFS fs, const char* path, const char *owner, const char *group)
@@ -1863,33 +1778,56 @@ int hdfsChown(hdfsFS fs, const char* pat
     }
 
     jobject jFS = (jobject)fs;
+    jobject jPath = NULL;
+    jstring jOwner = NULL, jGroup = NULL;
+    jthrowable jthr;
+    int ret;
 
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
-        return -1;
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsChown(path=%s): constructNewObjectOfPath", path);
+        goto done;
     }
 
-    jstring jOwnerString = owner ? (*env)->NewStringUTF(env, owner) : NULL;
-    jstring jGroupString = group ? (*env)->NewStringUTF(env, group) : NULL;
+    jthr = newJavaStr(env, owner, &jOwner); 
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsChown(path=%s): newJavaStr(%s)", path, owner);
+        goto done;
+    }
+    jthr = newJavaStr(env, group, &jGroup);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsChown(path=%s): newJavaStr(%s)", path, group);
+        goto done;
+    }
 
     //Create the directory
-    int ret = 0;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "setOwner", JMETHOD3(JPARAM(HADOOP_PATH), JPARAM(JAVA_STRING), JPARAM(JAVA_STRING), JAVA_VOID),
-                     jPath, jOwnerString, jGroupString) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::setOwner");
-        ret = -1;
+    jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
+            "setOwner", JMETHOD3(JPARAM(HADOOP_PATH), 
+                    JPARAM(JAVA_STRING), JPARAM(JAVA_STRING), JAVA_VOID),
+            jPath, jOwner, jGroup);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr,
+            NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
+            NOPRINT_EXC_UNRESOLVED_LINK,
+            "hdfsChown(path=%s, owner=%s, group=%s): "
+            "FileSystem#setOwner", path, owner, group);
         goto done;
     }
+    ret = 0;
 
- done:
+done:
     destroyLocalReference(env, jPath);
-    destroyLocalReference(env, jOwnerString);
-    destroyLocalReference(env, jGroupString);
+    destroyLocalReference(env, jOwner);
+    destroyLocalReference(env, jGroup);
 
-    return ret;
+    if (ret) {
+        errno = ret;
+        return -1;
+    }
+    return 0;
 }
 
 int hdfsChmod(hdfsFS fs, const char* path, short mode)
@@ -1905,49 +1843,58 @@ int hdfsChmod(hdfsFS fs, const char* pat
       return -1;
     }
 
+    jthrowable jthr;
+    jobject jPath = NULL, jPermObj = NULL;
     jobject jFS = (jobject)fs;
 
     // construct jPerm = FsPermission.createImmutable(short mode);
-
     jshort jmode = mode;
-
-    jobject jPermObj =
-      constructNewObjectOfClass(env, NULL, HADOOP_FSPERM,"(S)V",jmode);
-    if (jPermObj == NULL) {
-      errno = EINTERNAL;
-      return -1;
+    jthr = constructNewObjectOfClass(env, &jPermObj,
+                HADOOP_FSPERM,"(S)V",jmode);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "constructNewObjectOfClass(%s)", HADOOP_FSPERM);
+        return -1;
     }
 
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
-      destroyLocalReference(env, jPermObj);
-      return -1;
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsChmod(%s): constructNewObjectOfPath", path);
+        goto done;
     }
 
     //Create the directory
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "setPermission", JMETHOD2(JPARAM(HADOOP_PATH), JPARAM(HADOOP_FSPERM), JAVA_VOID),
-                     jPath, jPermObj) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::setPermission");
-        ret = -1;
+    jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
+            "setPermission",
+            JMETHOD2(JPARAM(HADOOP_PATH), JPARAM(HADOOP_FSPERM), JAVA_VOID),
+            jPath, jPermObj);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr,
+            NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
+            NOPRINT_EXC_UNRESOLVED_LINK,
+            "hdfsChmod(%s): FileSystem#setPermission", path);
         goto done;
     }
     ret = 0;
 
- done:
+done:
     destroyLocalReference(env, jPath);
     destroyLocalReference(env, jPermObj);
 
-    return ret;
+    if (ret) {
+        errno = ret;
+        return -1;
+    }
+    return 0;
 }
 
 int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime)
 {
     // JAVA EQUIVALENT:
     //  fs.setTimes(src, mtime, atime)
+    jthrowable jthr;
 
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
@@ -1959,41 +1906,46 @@ int hdfsUtime(hdfsFS fs, const char* pat
     jobject jFS = (jobject)fs;
 
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
-      fprintf(stderr, "could not construct path object\n");
-      return -1;
+    jobject jPath;
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsUtime(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);
 
-    int ret = 0;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, NULL, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "setTimes", JMETHOD3(JPARAM(HADOOP_PATH), "J", "J", JAVA_VOID),
-                     jPath, jmtime, jatime) != 0) {
-      fprintf(stderr, "call to setTime failed\n");
-      errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                 "FileSystem::setTimes");
-      ret = -1;
-      goto done;
-    }
-
- done:
+    jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
+            "setTimes", JMETHOD3(JPARAM(HADOOP_PATH), "J", "J", JAVA_VOID),
+            jPath, jmtime, jatime);
     destroyLocalReference(env, jPath);
-    return ret;
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr,
+            NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
+            NOPRINT_EXC_UNRESOLVED_LINK,
+            "hdfsUtime(path=%s): FileSystem#setTimes", path);
+        return -1;
+    }
+    return 0;
 }
 
-
-
-
 char***
 hdfsGetHosts(hdfsFS fs, const char* path, tOffset start, tOffset length)
 {
     // JAVA EQUIVALENT:
     //  fs.getFileBlockLoctions(new Path(path), start, length);
+    jthrowable jthr;
+    jobject jPath = NULL;
+    jobject jFileStatus = NULL;
+    jvalue jFSVal, jVal;
+    jobjectArray jBlockLocations = NULL, jFileBlockHosts = NULL;
+    jstring jHost = NULL;
+    char*** blockHosts = NULL;
+    int i, j, ret;
+    jsize jNumFileBlocks = 0;
 
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
@@ -2005,117 +1957,133 @@ hdfsGetHosts(hdfsFS fs, const char* path
     jobject jFS = (jobject)fs;
 
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath = constructNewObjectOfPath(env, path);
-    if (jPath == NULL) {
-        return NULL;
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsGetHosts(path=%s): constructNewObjectOfPath", path);
+        goto done;
     }
-
-    jvalue jFSVal;
-    jthrowable jFSExc = NULL;
-    if (invokeMethod(env, &jFSVal, &jFSExc, INSTANCE, jFS,
-                     HADOOP_FS, "getFileStatus", 
-                     "(Lorg/apache/hadoop/fs/Path;)"
-                     "Lorg/apache/hadoop/fs/FileStatus;",
-                     jPath) != 0) {
-        errno = errnoFromException(jFSExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::getFileStatus");
+    jthr = invokeMethod(env, &jFSVal, INSTANCE, jFS,
+            HADOOP_FS, "getFileStatus", "(Lorg/apache/hadoop/fs/Path;)"
+            "Lorg/apache/hadoop/fs/FileStatus;", jPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, NOPRINT_EXC_FILE_NOT_FOUND,
+                "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
+                "FileSystem#getFileStatus", path, start, length);
         destroyLocalReference(env, jPath);
-        return NULL;
+        goto done;
     }
-    jobject jFileStatus = jFSVal.l;
+    jFileStatus = jFSVal.l;
 
-    //org.apache.hadoop.fs.FileSystem::getFileBlockLocations
-    char*** blockHosts = NULL;
-    jobjectArray jBlockLocations;;
-    jvalue jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS,
+    //org.apache.hadoop.fs.FileSystem#getFileBlockLocations
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
                      HADOOP_FS, "getFileBlockLocations", 
                      "(Lorg/apache/hadoop/fs/FileStatus;JJ)"
                      "[Lorg/apache/hadoop/fs/BlockLocation;",
-                     jFileStatus, start, length) != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::getFileBlockLocations");
-        destroyLocalReference(env, jPath);
-        destroyLocalReference(env, jFileStatus);
-        return NULL;
+                     jFileStatus, start, length);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
+                "FileSystem#getFileBlockLocations", path, start, length);
+        goto done;
     }
     jBlockLocations = jVal.l;
 
     //Figure out no of entries in jBlockLocations
     //Allocate memory and add NULL at the end
-    jsize jNumFileBlocks = (*env)->GetArrayLength(env, jBlockLocations);
+    jNumFileBlocks = (*env)->GetArrayLength(env, jBlockLocations);
 
-    blockHosts = malloc(sizeof(char**) * (jNumFileBlocks+1));
+    blockHosts = calloc(jNumFileBlocks + 1, sizeof(char**));
     if (blockHosts == NULL) {
-        errno = ENOMEM;
+        ret = ENOMEM;
         goto done;
     }
-    blockHosts[jNumFileBlocks] = NULL;
     if (jNumFileBlocks == 0) {
-        errno = 0;
+        ret = 0;
         goto done;
     }
 
     //Now parse each block to get hostnames
-    int i = 0;
-    for (i=0; i < jNumFileBlocks; ++i) {
+    for (i = 0; i < jNumFileBlocks; ++i) {
         jobject jFileBlock =
             (*env)->GetObjectArrayElement(env, jBlockLocations, i);
-        
-        jvalue jVal;
-        jobjectArray jFileBlockHosts;
-        if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFileBlock, HADOOP_BLK_LOC,
-                         "getHosts", "()[Ljava/lang/String;") ||
-                jVal.l == NULL) {
-            errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                       "BlockLocation::getHosts");
-            destroyLocalReference(env, jPath);
-            destroyLocalReference(env, jFileStatus);
-            destroyLocalReference(env, jBlockLocations);
-            return NULL;
+        if (!jFileBlock) {
+            ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+                "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
+                "GetObjectArrayElement(%d)", path, start, length, i);
+            goto done;
         }
         
+        jthr = invokeMethod(env, &jVal, INSTANCE, jFileBlock, HADOOP_BLK_LOC,
+                         "getHosts", "()[Ljava/lang/String;");
+        if (jthr) {
+            ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
+                "BlockLocation#getHosts", path, start, length);
+            goto done;
+        }
         jFileBlockHosts = jVal.l;
-        //Figure out no of hosts in jFileBlockHosts
-        //Allocate memory and add NULL at the end
+        if (!jFileBlockHosts) {
+            fprintf(stderr,
+                "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
+                "BlockLocation#getHosts returned NULL", path, start, length);
+            ret = EINTERNAL;
+            goto done;
+        }
+        //Figure out no of hosts in jFileBlockHosts, and allocate the memory
         jsize jNumBlockHosts = (*env)->GetArrayLength(env, jFileBlockHosts);
-        blockHosts[i] = malloc(sizeof(char*) * (jNumBlockHosts+1));
-        if (blockHosts[i] == NULL) {
-            int x = 0;
-            for (x=0; x < i; ++x) {
-                free(blockHosts[x]);
-            }
-            free(blockHosts);
-            errno = ENOMEM;
+        blockHosts[i] = calloc(jNumBlockHosts + 1, sizeof(char*));
+        if (!blockHosts[i]) {
+            ret = ENOMEM;
             goto done;
         }
-        blockHosts[i][jNumBlockHosts] = NULL;
 
         //Now parse each hostname
-        int j = 0;
         const char *hostName;
-        for (j=0; j < jNumBlockHosts; ++j) {
-            jstring jHost =
-                (*env)->GetObjectArrayElement(env, jFileBlockHosts, j);
-           
+        for (j = 0; j < jNumBlockHosts; ++j) {
+            jHost = (*env)->GetObjectArrayElement(env, jFileBlockHosts, j);
+            if (!jHost) {
+                ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+                    "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"): "
+                    "NewByteArray", path, start, length);
+                goto done;
+            }
             hostName =
                 (const char*)((*env)->GetStringUTFChars(env, jHost, NULL));
+            if (!hostName) {
+                ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+                    "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64", "
+                    "j=%d out of %d): GetStringUTFChars",
+                    path, start, length, j, jNumBlockHosts);
+                goto done;
+            }
             blockHosts[i][j] = strdup(hostName);
-
             (*env)->ReleaseStringUTFChars(env, jHost, hostName);
+            if (!blockHosts[i][j]) {
+                ret = ENOMEM;
+                goto done;
+            }
             destroyLocalReference(env, jHost);
+            jHost = NULL;
         }
 
         destroyLocalReference(env, jFileBlockHosts);
+        jFileBlockHosts = NULL;
     }
-  
-    done:
+    ret = 0;
 
-    //Delete unnecessary local references
+done:
     destroyLocalReference(env, jPath);
     destroyLocalReference(env, jFileStatus);
     destroyLocalReference(env, jBlockLocations);
+    destroyLocalReference(env, jFileBlockHosts);
+    destroyLocalReference(env, jHost);
+    if (ret) {
+        if (blockHosts) {
+            hdfsFreeHosts(blockHosts);
+        }
+        return NULL;
+    }
 
     return blockHosts;
 }
@@ -2148,19 +2116,17 @@ tOffset hdfsGetDefaultBlockSize(hdfsFS f
 
     jobject jFS = (jobject)fs;
 
-    //FileSystem::getDefaultBlockSize()
-    tOffset blockSize = -1;
+    //FileSystem#getDefaultBlockSize()
     jvalue jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "getDefaultBlockSize", "()J") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::getDefaultBlockSize");
+    jthrowable jthr;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                     "getDefaultBlockSize", "()J");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsGetDefaultBlockSize: FileSystem#getDefaultBlockSize");
         return -1;
     }
-    blockSize = jVal.j;
-
-    return blockSize;
+    return jVal.j;
 }
 
 
@@ -2180,24 +2146,25 @@ tOffset hdfsGetCapacity(hdfsFS fs)
 
     jobject jFS = (jobject)fs;
 
-    //FileSystem::getStatus
+    //FileSystem#getStatus
     jvalue  jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::getStatus");
+    jthrowable jthr;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                     "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsGetCapacity: FileSystem#getStatus");
         return -1;
     }
     jobject fss = (jobject)jVal.l;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, fss, HADOOP_FSSTATUS,
-                     "getCapacity", "()J") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FsStatus::getCapacity");
-        destroyLocalReference(env, fss);
+    jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,
+                     "getCapacity", "()J");
+    destroyLocalReference(env, fss);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsGetCapacity: FsStatus#getCapacity");
         return -1;
     }
-    destroyLocalReference(env, fss);
     return jVal.j;
 }
 
@@ -2218,166 +2185,163 @@ tOffset hdfsGetUsed(hdfsFS fs)
 
     jobject jFS = (jobject)fs;
 
-    //FileSystem::getStatus
+    //FileSystem#getStatus
     jvalue  jVal;
-    jthrowable jExc = NULL;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FileSystem::getStatus");
+    jthrowable jthr;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                     "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsGetUsed: FileSystem#getStatus");
         return -1;
     }
     jobject fss = (jobject)jVal.l;
-    if (invokeMethod(env, &jVal, &jExc, INSTANCE, fss, HADOOP_FSSTATUS,
-                     "getUsed", "()J") != 0) {
-        errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
-                                   "FsStatus::getUsed");
-        destroyLocalReference(env, fss);
+    jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,

[... 484 lines stripped ...]