diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 90729b6fde5..35a5f5b5f27 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -557,6 +557,8 @@ Branch-2 ( Unreleased changes ) HDFS-3719. Re-enable append-related tests in TestFileConcurrentReader. (Andrew Wang via atm) + HDFS-3579. libhdfs: fix exception handling. (Colin Patrick McCabe via atm) + BREAKDOWN OF HDFS-3042 SUBTASKS HDFS-2185. HDFS portion of ZK-based FailoverController (todd) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt index 7c1441fca52..0cbd1cfc6fa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt @@ -95,6 +95,7 @@ set(_FUSE_DFS_VERSION 0.1.0) CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h) add_dual_library(hdfs + main/native/libhdfs/exception.c main/native/libhdfs/hdfs.c main/native/libhdfs/jni_helper.c ) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c index 70468f80c02..63de10ac99a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c @@ -16,9 +16,11 @@ * limitations under the License. */ +#include "exception.h" #include "hdfs.h" #include "jni_helper.h" +#include #include #include @@ -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 *env, jobject jObject) * 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) { + jthrowable jthr; + jstring jPathString; + jobject jPath; + //Construct a java.lang.String object - jstring jPathString = (*env)->NewStringUTF(env, path); - + 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,175 +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; + destroyLocalReference(env, jkey); + destroyLocalReference(env, jvalue); + return jthr; } -/** - * 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; -} - -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); -done: - destroyLocalReference(env, jConfiguration); - if (ret) - errno = ret; - return ret; -} - -static int hadoopConfGetInt(JNIEnv *env, jobject jConfiguration, - const char *key, int32_t *val) -{ - int ret; - jthrowable jExc = 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, - 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); - } - *val = jVal.i; - return 0; -} - -int hdfsConfGetInt(const char *key, int32_t *val) -{ - JNIEnv *env; - int ret; - 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; + jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V"); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsConfGetStr(%s): new Configuration", key); goto done; } - ret = hadoopConfGetInt(env, jConfiguration, key, val); + 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) @@ -404,6 +227,58 @@ void hdfsConfStrFree(char *val) free(val); } +static jthrowable hadoopConfGetInt(JNIEnv *env, jobject jConfiguration, + const char *key, int32_t *val) +{ + jthrowable jthr = NULL; + jvalue jVal; + jstring jkey = NULL; + + 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 (jthr) + return jthr; + *val = jVal.i; + return NULL; +} + +int hdfsConfGetInt(const char *key, int32_t *val) +{ + JNIEnv *env; + int ret; + jobject jConfiguration = NULL; + jthrowable jthr; + + env = getJNIEnv(); + if (env == NULL) { + ret = EINTERNAL; + goto done; + } + 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 = 0; +done: + destroyLocalReference(env, jConfiguration); + if (ret) + errno = ret; + return ret; +} + struct hdfsBuilder { int forceNewInstance; const char *nn; @@ -551,54 +426,72 @@ static int calcEffectiveURI(struct hdfsBuilder *bld, char ** uri) 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 hdfsBuilder *bld) } 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; } } + 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: - if (jFS) { - /* Create a global reference for this fs */ - gFsRef = (*env)->NewGlobalRef(env, jFS); - } - // 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"); + 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; + } + (*env)->DeleteGlobalRef(env, jFS); + if (ret) { + errno = ret; return -1; } - - //Release unnecessary references - (*env)->DeleteGlobalRef(env, jFS); - return 0; } @@ -740,7 +663,14 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags, 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 char* path, int flags, 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 char* path, int flags, //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 char* path, int flags, //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 char* path, int flags, // 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; + 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; - 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) { + 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 file) 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 file) 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, hdfsFS fs, hdfsFile f, 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,25 +988,43 @@ tSize hdfsRead(hdfsFS fs, hdfsFile f, void* buffer, tSize length) } jbyteArray jbRarray; - jint noReadBytes = 0; + jint noReadBytes = length; jvalue jVal; - jthrowable jExc = NULL; + jthrowable jthr; //Read the requisite bytes jbRarray = (*env)->NewByteArray(env, length); - - int success = invokeMethod(env, &jVal, &jExc, 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 (!jbRarray) { + errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "hdfsRead: NewByteArray"); + return -1; } + jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream, HADOOP_ISTRM, + "read", "([B)I", jbRarray); + 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 @@ -1096,63 +1046,53 @@ tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length) 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, tOffset position, 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; + if (!jbRarray) { + errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "hdfsPread: NewByteArray"); + return -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; + 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, const void* buffer, tSize length) 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, const void* buffer, tSize length) 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; - } - destroyLocalReference(env, jbWarray); + if (length < 0) { + errno = EINVAL; + return -1; } - - //Return no. of bytes succesfully written (libc way) - //i.e. 'length' itself! ;-) + 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; + } + 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, tOffset desiredPos) 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* src, hdfsFS dstFS, const char* dst) //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 = 0; //Only copy + jobject jConfiguration = NULL, jSrcPath = NULL, jDstPath = NULL; + jthrowable jthr; 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; + int ret; + + jthr = constructNewObjectOfPath(env, src, &jSrcPath); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsCopyImpl(src=%s): constructNewObjectOfPath", src); + goto done; + } + jthr = constructNewObjectOfPath(env, dst, &jDstPath); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsCopyImpl(dst=%s): constructNewObjectOfPath", dst); goto done; } - done: + //Create the org.apache.hadoop.conf.Configuration object + jthr = constructNewObjectOfClass(env, &jConfiguration, + HADOOP_CONF, "()V"); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsCopyImpl: Configuration constructor"); + goto done; + } - //Delete unnecessary local references + //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: 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* path, int recursive) } jobject jFS = (jobject)fs; - - //Create an object of java.io.File - jobject jPath = constructNewObjectOfPath(env, path); - if (jPath == NULL) { - return -1; - } - - //Delete the file + jthrowable jthr; + jobject jPath; jvalue jVal; - jthrowable jExc = NULL; - jboolean jRecursive = recursive ? JNI_TRUE : JNI_FALSE; - if (invokeMethod(env, &jVal, &jExc, 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); + + jthr = constructNewObjectOfPath(env, path, &jPath); + if (jthr) { + errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsDelete(path=%s): constructNewObjectOfPath", path); return -1; } - - //Delete unnecessary local references + jboolean jRecursive = recursive ? JNI_TRUE : JNI_FALSE; + jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS, + "delete", "(Lorg/apache/hadoop/fs/Path;Z)Z", + jPath, jRecursive); destroyLocalReference(env, jPath); - - return (jVal.z) ? 0 : -1; + if (jthr) { + errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsDelete(path=%s, recursive=%d): " + "FileSystem#delete", path, recursive); + return -1; + } + if (!jVal.z) { + errno = EIO; + return -1; + } + return 0; } @@ -1632,40 +1501,45 @@ int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath) } jobject jFS = (jobject)fs; - - //Create objects of org.apache.hadoop.fs.Path - jobject jOldPath = NULL; - jobject jNewPath = NULL; - - jOldPath = constructNewObjectOfPath(env, oldPath); - if (jOldPath == NULL) { - return -1; - } - - jNewPath = constructNewObjectOfPath(env, newPath); - if (jNewPath == NULL) { - destroyLocalReference(env, jOldPath); - return -1; - } - - //Rename the file + jthrowable jthr; + jobject jOldPath = NULL, jNewPath = NULL; + int ret = -1; jvalue jVal; - jthrowable jExc = NULL; - if (invokeMethod(env, &jVal, &jExc, 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; + + jthr = constructNewObjectOfPath(env, oldPath, &jOldPath ); + if (jthr) { + errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsRename: constructNewObjectOfPath(%s)", oldPath); + goto done; + } + jthr = constructNewObjectOfPath(env, newPath, &jNewPath); + if (jthr) { + errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsRename: constructNewObjectOfPath(%s)", newPath); + goto done; } - //Delete unnecessary local references + // 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); + 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; + +done: destroyLocalReference(env, jOldPath); destroyLocalReference(env, jNewPath); - - return (jVal.z) ? 0 : -1; + return ret; } @@ -1683,45 +1557,67 @@ char* hdfsGetWorkingDirectory(hdfsFS fs, char* buffer, size_t bufferSize) 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); + ret = snprintf(buffer, bufferSize, "%s", jPathChars); + if (ret >= bufferSize) { + ret = ENAMETOOLONG; + goto done; + } + ret = 0; - //Delete unnecessary local references - (*env)->ReleaseStringUTFChars(env, jPathString, jPathChars); - - 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, const char* path) } 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 char* path) } 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 char* path, int16_t replication) } 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; + 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; } - done: - - //Delete unnecessary local references - destroyLocalReference(env, jPath); - - 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* path, const char *owner, const char *group) } 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; - } - - jstring jOwnerString = owner ? (*env)->NewStringUTF(env, owner) : NULL; - jstring jGroupString = group ? (*env)->NewStringUTF(env, group) : NULL; - - //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 = constructNewObjectOfPath(env, path, &jPath); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsChown(path=%s): constructNewObjectOfPath", path); goto done; } - done: - destroyLocalReference(env, jPath); - destroyLocalReference(env, jOwnerString); - destroyLocalReference(env, jGroupString); + 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; + } - return ret; + //Create the directory + 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: + destroyLocalReference(env, jPath); + destroyLocalReference(env, jOwner); + destroyLocalReference(env, jGroup); + + 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* path, short mode) 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* path, tTime mtime, tTime atime) 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, tOffset start, tOffset length) 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; - } - - jFileBlockHosts = jVal.l; - //Figure out no of hosts in jFileBlockHosts - //Allocate memory and add NULL at the end - 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; + 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; + 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] = 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 fs) 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, + "getUsed", "()J"); + destroyLocalReference(env, fss); + if (jthr) { + errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsGetUsed: FsStatus#getUsed"); return -1; } - destroyLocalReference(env, fss); return jVal.j; - } -static int +static jthrowable getFileInfoFromStat(JNIEnv *env, jobject jStat, hdfsFileInfo *fileInfo) { jvalue jVal; - jthrowable jExc = NULL; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, - HADOOP_STAT, "isDir", "()Z") != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileStatus::isDir"); - return -1; - } + jthrowable jthr; + jobject jPath = NULL; + jstring jPathName = NULL; + jstring jUserName = NULL; + jstring jGroupName = NULL; + jobject jPermission = NULL; + + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, + HADOOP_STAT, "isDir", "()Z"); + if (jthr) + goto done; fileInfo->mKind = jVal.z ? kObjectKindDirectory : kObjectKindFile; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, - HADOOP_STAT, "getReplication", "()S") != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileStatus::getReplication"); - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, + HADOOP_STAT, "getReplication", "()S"); + if (jthr) + goto done; fileInfo->mReplication = jVal.s; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, - HADOOP_STAT, "getBlockSize", "()J") != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileStatus::getBlockSize"); - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, + HADOOP_STAT, "getBlockSize", "()J"); + if (jthr) + goto done; fileInfo->mBlockSize = jVal.j; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, - HADOOP_STAT, "getModificationTime", "()J") != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileStatus::getModificationTime"); - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, + HADOOP_STAT, "getModificationTime", "()J"); + if (jthr) + goto done; fileInfo->mLastMod = jVal.j / 1000; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, - HADOOP_STAT, "getAccessTime", "()J") != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileStatus::getAccessTime"); - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, + HADOOP_STAT, "getAccessTime", "()J"); + if (jthr) + goto done; fileInfo->mLastAccess = (tTime) (jVal.j / 1000); - if (fileInfo->mKind == kObjectKindFile) { - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, - HADOOP_STAT, "getLen", "()J") != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileStatus::getLen"); - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, + HADOOP_STAT, "getLen", "()J"); + if (jthr) + goto done; fileInfo->mSize = jVal.j; } - jobject jPath; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, HADOOP_STAT, - "getPath", "()Lorg/apache/hadoop/fs/Path;") || - jVal.l == NULL) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "Path::getPath"); - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT, + "getPath", "()Lorg/apache/hadoop/fs/Path;"); + if (jthr) + goto done; jPath = jVal.l; - - jstring jPathName; - const char *cPathName; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jPath, HADOOP_PATH, - "toString", "()Ljava/lang/String;")) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "Path::toString"); - destroyLocalReference(env, jPath); - return -1; + if (jPath == NULL) { + jthr = newRuntimeError(env, "org.apache.hadoop.fs.FileStatus#" + "getPath returned NULL!"); + goto done; } + + jthr = invokeMethod(env, &jVal, INSTANCE, jPath, HADOOP_PATH, + "toString", "()Ljava/lang/String;"); + if (jthr) + goto done; jPathName = jVal.l; - cPathName = (const char*) ((*env)->GetStringUTFChars(env, jPathName, NULL)); + const char *cPathName = + (const char*) ((*env)->GetStringUTFChars(env, jPathName, NULL)); + if (!cPathName) { + jthr = getPendingExceptionAndClear(env); + goto done; + } fileInfo->mName = strdup(cPathName); (*env)->ReleaseStringUTFChars(env, jPathName, cPathName); - destroyLocalReference(env, jPath); - destroyLocalReference(env, jPathName); - jstring jUserName; - const char* cUserName; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, HADOOP_STAT, - "getOwner", "()Ljava/lang/String;")) { - fprintf(stderr, "Call to org.apache.hadoop.fs." - "FileStatus::getOwner failed!\n"); - errno = EINTERNAL; - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT, + "getOwner", "()Ljava/lang/String;"); + if (jthr) + goto done; jUserName = jVal.l; - cUserName = (const char*) ((*env)->GetStringUTFChars(env, jUserName, NULL)); + const char* cUserName = + (const char*) ((*env)->GetStringUTFChars(env, jUserName, NULL)); + if (!cUserName) { + jthr = getPendingExceptionAndClear(env); + goto done; + } fileInfo->mOwner = strdup(cUserName); (*env)->ReleaseStringUTFChars(env, jUserName, cUserName); - destroyLocalReference(env, jUserName); - jstring jGroupName; const char* cGroupName; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, HADOOP_STAT, - "getGroup", "()Ljava/lang/String;")) { - fprintf(stderr, "Call to org.apache.hadoop.fs." - "FileStatus::getGroup failed!\n"); - errno = EINTERNAL; - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT, + "getGroup", "()Ljava/lang/String;"); + if (jthr) + goto done; jGroupName = jVal.l; cGroupName = (const char*) ((*env)->GetStringUTFChars(env, jGroupName, NULL)); + if (!cGroupName) { + jthr = getPendingExceptionAndClear(env); + goto done; + } fileInfo->mGroup = strdup(cGroupName); (*env)->ReleaseStringUTFChars(env, jGroupName, cGroupName); - destroyLocalReference(env, jGroupName); - jobject jPermission; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat, HADOOP_STAT, - "getPermission", "()Lorg/apache/hadoop/fs/permission/FsPermission;") || - jVal.l == NULL) { - fprintf(stderr, "Call to org.apache.hadoop.fs." - "FileStatus::getPermission failed!\n"); - errno = EINTERNAL; - return -1; + jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT, + "getPermission", + "()Lorg/apache/hadoop/fs/permission/FsPermission;"); + if (jthr) + goto done; + if (jVal.l == NULL) { + jthr = newRuntimeError(env, "%s#getPermission returned NULL!", + HADOOP_STAT); + goto done; } jPermission = jVal.l; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jPermission, HADOOP_FSPERM, - "toShort", "()S") != 0) { - fprintf(stderr, "Call to org.apache.hadoop.fs.permission." - "FsPermission::toShort failed!\n"); - errno = EINTERNAL; - return -1; - } + jthr = invokeMethod(env, &jVal, INSTANCE, jPermission, HADOOP_FSPERM, + "toShort", "()S"); + if (jthr) + goto done; fileInfo->mPermissions = jVal.s; - destroyLocalReference(env, jPermission); + jthr = NULL; - return 0; +done: + if (jthr) + hdfsFreeFileInfoEntry(fileInfo); + destroyLocalReference(env, jPath); + destroyLocalReference(env, jPathName); + destroyLocalReference(env, jUserName); + destroyLocalReference(env, jGroupName); + destroyLocalReference(env, jPermission); + destroyLocalReference(env, jPath); + return jthr; } -static int -getFileInfo(JNIEnv *env, jobject jFS, jobject jPath, hdfsFileInfo *fileInfo) +static jthrowable +getFileInfo(JNIEnv *env, jobject jFS, jobject jPath, hdfsFileInfo **fileInfo) { // JAVA EQUIVALENT: // fs.isDirectory(f) @@ -2388,35 +2352,33 @@ getFileInfo(JNIEnv *env, jobject jFS, jobject jPath, hdfsFileInfo *fileInfo) // f.getOwner() // f.getGroup() // f.getPermission().toShort() - jobject jStat; jvalue jVal; - jthrowable jExc = NULL; + jthrowable jthr; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS, + jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS, "exists", JMETHOD1(JPARAM(HADOOP_PATH), "Z"), - jPath) != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileSystem::exists"); - return -1; - } - + jPath); + if (jthr) + return jthr; if (jVal.z == 0) { - errno = ENOENT; - return -1; - } - - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS, - "getFileStatus", JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_STAT)), - jPath) != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileSystem::getFileStatus"); - return -1; + *fileInfo = NULL; + return NULL; } + jthr = invokeMethod(env, &jVal, INSTANCE, jFS, + HADOOP_FS, "getFileStatus", + JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_STAT)), jPath); + if (jthr) + return jthr; jStat = jVal.l; - int ret = getFileInfoFromStat(env, jStat, fileInfo); + *fileInfo = calloc(1, sizeof(hdfsFileInfo)); + if (!*fileInfo) { + destroyLocalReference(env, jStat); + return newRuntimeError(env, "getFileInfo: OOM allocating hdfsFileInfo"); + } + jthr = getFileInfoFromStat(env, jStat, *fileInfo); destroyLocalReference(env, jStat); - return ret; + return jthr; } @@ -2428,6 +2390,13 @@ hdfsFileInfo* hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries) // Path []pathList = fs.listPaths(p) // foreach path in pathList // getFileInfo(path) + jthrowable jthr; + jobject jPath = NULL; + hdfsFileInfo *pathList = NULL; + jobjectArray jPathList = NULL; + jvalue jVal; + jsize jPathListSize = 0; + int ret; //Get the JNIEnv* corresponding to current thread JNIEnv* env = getJNIEnv(); @@ -2439,38 +2408,36 @@ hdfsFileInfo* hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries) 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, + "hdfsListDirectory(%s): constructNewObjectOfPath", path); + goto done; } - hdfsFileInfo *pathList = 0; - - jobjectArray jPathList = NULL; - jvalue jVal; - jthrowable jExc = NULL; - if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_DFS, "listStatus", + jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_DFS, "listStatus", JMETHOD1(JPARAM(HADOOP_PATH), JARRPARAM(HADOOP_STAT)), - jPath) != 0) { - errno = errnoFromException(jExc, env, "org.apache.hadoop.fs." - "FileSystem::listStatus"); - destroyLocalReference(env, jPath); - return NULL; + jPath); + if (jthr) { + ret = printExceptionAndFree(env, jthr, + NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND | + NOPRINT_EXC_UNRESOLVED_LINK, + "hdfsListDirectory(%s): FileSystem#listStatus", path); + goto done; } jPathList = jVal.l; - //Figure out no of entries in that directory - jsize jPathListSize = (*env)->GetArrayLength(env, jPathList); - *numEntries = jPathListSize; + //Figure out the number of entries in that directory + jPathListSize = (*env)->GetArrayLength(env, jPathList); if (jPathListSize == 0) { - errno = 0; + ret = 0; goto done; } //Allocate memory pathList = calloc(jPathListSize, sizeof(hdfsFileInfo)); if (pathList == NULL) { - errno = ENOMEM; + ret = ENOMEM; goto done; } @@ -2479,21 +2446,33 @@ hdfsFileInfo* hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries) jobject tmpStat; for (i=0; i < jPathListSize; ++i) { tmpStat = (*env)->GetObjectArrayElement(env, jPathList, i); - if (getFileInfoFromStat(env, tmpStat, &pathList[i])) { - hdfsFreeFileInfo(pathList, jPathListSize); - destroyLocalReference(env, tmpStat); - pathList = NULL; + if (!tmpStat) { + ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "hdfsListDirectory(%s): GetObjectArrayElement(%d out of %d)", + path, i, jPathListSize); goto done; } + jthr = getFileInfoFromStat(env, tmpStat, &pathList[i]); destroyLocalReference(env, tmpStat); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsListDirectory(%s): getFileInfoFromStat(%d out of %d)", + path, i, jPathListSize); + goto done; + } } + ret = 0; - done: - - //Delete unnecessary local references +done: destroyLocalReference(env, jPath); destroyLocalReference(env, jPathList); + if (ret) { + hdfsFreeFileInfo(pathList, jPathListSize); + errno = ret; + return NULL; + } + *numEntries = jPathListSize; return pathList; } @@ -2518,42 +2497,44 @@ hdfsFileInfo *hdfsGetPathInfo(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) { + jobject jPath; + jthrowable jthr = constructNewObjectOfPath(env, path, &jPath); + if (jthr) { + errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsGetPathInfo(%s): constructNewObjectOfPath", path); return NULL; } - - hdfsFileInfo *fileInfo = calloc(1, sizeof(hdfsFileInfo)); - if (getFileInfo(env, jFS, jPath, fileInfo)) { - hdfsFreeFileInfo(fileInfo, 1); - fileInfo = NULL; - goto done; - } - - done: - - //Delete unnecessary local references + hdfsFileInfo *fileInfo; + jthr = getFileInfo(env, jFS, jPath, &fileInfo); destroyLocalReference(env, jPath); - + if (jthr) { + errno = printExceptionAndFree(env, jthr, + NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND | + NOPRINT_EXC_UNRESOLVED_LINK, + "hdfsGetPathInfo(%s): getFileInfo", path); + return NULL; + } + if (!fileInfo) { + errno = ENOENT; + return NULL; + } return fileInfo; } - +static void hdfsFreeFileInfoEntry(hdfsFileInfo *hdfsFileInfo) +{ + free(hdfsFileInfo->mName); + free(hdfsFileInfo->mOwner); + free(hdfsFileInfo->mGroup); + memset(hdfsFileInfo, 0, sizeof(hdfsFileInfo)); +} void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries) { //Free the mName, mOwner, and mGroup int i; for (i=0; i < numEntries; ++i) { - if (hdfsFileInfo[i].mName) { - free(hdfsFileInfo[i].mName); - } - if (hdfsFileInfo[i].mOwner) { - free(hdfsFileInfo[i].mOwner); - } - if (hdfsFileInfo[i].mGroup) { - free(hdfsFileInfo[i].mGroup); - } + hdfsFreeFileInfoEntry(hdfsFileInfo + i); } //Free entire block diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h index 4f252cd8d0d..e32c299c76f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h @@ -252,7 +252,9 @@ extern "C" { * hdfsDisconnect - Disconnect from the hdfs file system. * Disconnect from hdfs. * @param fs The configured filesystem handle. - * @return Returns 0 on success, -1 on error. + * @return Returns 0 on success, -1 on error. + * Even if there is an error, the resources associated with the + * hdfsFS will be freed. */ int hdfsDisconnect(hdfsFS fs); @@ -280,6 +282,10 @@ extern "C" { * @param fs The configured filesystem handle. * @param file The file handle. * @return Returns 0 on success, -1 on error. + * On error, errno will be set appropriately. + * If the hdfs file was valid, the memory associated with it will + * be freed at the end of this call, even if there was an I/O + * error. */ int hdfsCloseFile(hdfsFS fs, hdfsFile file); @@ -336,8 +342,7 @@ extern "C" { * @param position Position from which to read * @param buffer The buffer to copy read bytes into. * @param length The length of the buffer. - * @return Returns the number of bytes actually read, possibly less than - * than length;-1 on error. + * @return See hdfsRead */ tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position, void* buffer, tSize length); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c index d788f6282ee..bcd22882208 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c @@ -17,6 +17,7 @@ */ #include "config.h" +#include "exception.h" #include "jni_helper.h" #include @@ -85,16 +86,57 @@ static void hdfsThreadDestructor(void *v) free(tls); } - -static int validateMethodType(MethType methType) +void destroyLocalReference(JNIEnv *env, jobject jObject) { - if (methType != STATIC && methType != INSTANCE) { - fprintf(stderr, "Unimplemented method type\n"); - return 0; - } - return 1; + if (jObject) + (*env)->DeleteLocalRef(env, jObject); } +static jthrowable validateMethodType(JNIEnv *env, MethType methType) +{ + if (methType != STATIC && methType != INSTANCE) { + return newRuntimeError(env, "validateMethodType(methType=%d): " + "illegal method type.\n", methType); + } + return NULL; +} + +jthrowable newJavaStr(JNIEnv *env, const char *str, jstring *out) +{ + jstring jstr; + + if (!str) { + /* Can't pass NULL to NewStringUTF: the result would be + * implementation-defined. */ + *out = NULL; + return NULL; + } + jstr = (*env)->NewStringUTF(env, str); + if (!jstr) { + /* If NewStringUTF returns NULL, an exception has been thrown, + * which we need to handle. Probaly an OOM. */ + return getPendingExceptionAndClear(env); + } + *out = jstr; + return NULL; +} + +jthrowable newCStr(JNIEnv *env, jstring jstr, char **out) +{ + const char *tmp; + + if (!jstr) { + *out = NULL; + return NULL; + } + tmp = (*env)->GetStringUTFChars(env, jstr, NULL); + if (!tmp) { + return getPendingExceptionAndClear(env); + } + *out = strdup(tmp); + (*env)->ReleaseStringUTFChars(env, jstr, tmp); + return NULL; +} static int hashTableInit(void) { @@ -156,7 +198,7 @@ static void* searchEntryFromTable(const char *key) -int invokeMethod(JNIEnv *env, RetVal *retval, Exc *exc, MethType methType, +jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType, jobject instObj, const char *className, const char *methName, const char *methSignature, ...) { @@ -167,21 +209,16 @@ int invokeMethod(JNIEnv *env, RetVal *retval, Exc *exc, MethType methType, const char *str; char returnType; - if (! validateMethodType(methType)) { - return -1; - } - cls = globalClassReference(className, env); - if (cls == NULL) { - return -2; - } - - mid = methodIdFromClass(className, methName, methSignature, - methType, env); - if (mid == NULL) { - (*env)->ExceptionDescribe(env); - return -3; - } - + jthr = validateMethodType(env, methType); + if (jthr) + return jthr; + jthr = globalClassReference(className, env, &cls); + if (jthr) + return jthr; + jthr = methodIdFromClass(className, methName, methSignature, + methType, env, &mid); + if (jthr) + return jthr; str = methSignature; while (*str != ')') str++; str++; @@ -248,43 +285,14 @@ int invokeMethod(JNIEnv *env, RetVal *retval, Exc *exc, MethType methType, va_end(args); jthr = (*env)->ExceptionOccurred(env); - if (jthr != NULL) { - if (exc != NULL) - *exc = jthr; - else - (*env)->ExceptionDescribe(env); - return -1; + if (jthr) { + (*env)->ExceptionClear(env); + return jthr; } - return 0; + return NULL; } -jarray constructNewArrayString(JNIEnv *env, Exc *exc, const char **elements, int size) { - const char *className = "java/lang/String"; - jobjectArray result; - int i; - jclass arrCls = (*env)->FindClass(env, className); - if (arrCls == NULL) { - fprintf(stderr, "could not find class %s\n",className); - return NULL; /* exception thrown */ - } - result = (*env)->NewObjectArray(env, size, arrCls, - NULL); - if (result == NULL) { - fprintf(stderr, "ERROR: could not construct new array\n"); - return NULL; /* out of memory error thrown */ - } - for (i = 0; i < size; i++) { - jstring jelem = (*env)->NewStringUTF(env,elements[i]); - if (jelem == NULL) { - fprintf(stderr, "ERROR: jelem == NULL\n"); - } - (*env)->SetObjectArrayElement(env, result, i, jelem); - (*env)->DeleteLocalRef(env, jelem); - } - return result; -} - -jobject constructNewObjectOfClass(JNIEnv *env, Exc *exc, const char *className, +jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out, const char *className, const char *ctorSignature, ...) { va_list args; @@ -293,50 +301,37 @@ jobject constructNewObjectOfClass(JNIEnv *env, Exc *exc, const char *className, jobject jobj; jthrowable jthr; - cls = globalClassReference(className, env); - if (cls == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; - } - - mid = methodIdFromClass(className, "", ctorSignature, - INSTANCE, env); - if (mid == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; - } + jthr = globalClassReference(className, env, &cls); + if (jthr) + return jthr; + jthr = methodIdFromClass(className, "", ctorSignature, + INSTANCE, env, &mid); + if (jthr) + return jthr; va_start(args, ctorSignature); jobj = (*env)->NewObjectV(env, cls, mid, args); va_end(args); - jthr = (*env)->ExceptionOccurred(env); - if (jthr != NULL) { - if (exc != NULL) - *exc = jthr; - else - (*env)->ExceptionDescribe(env); - } - return jobj; + if (!jobj) + return getPendingExceptionAndClear(env); + *out = jobj; + return NULL; } - - -jmethodID methodIdFromClass(const char *className, const char *methName, +jthrowable methodIdFromClass(const char *className, const char *methName, const char *methSignature, MethType methType, - JNIEnv *env) + JNIEnv *env, jmethodID *out) { - jclass cls = globalClassReference(className, env); - if (cls == NULL) { - fprintf(stderr, "could not find class %s\n", className); - return NULL; - } + jclass cls; + jthrowable jthr; + jthr = globalClassReference(className, env, &cls); + if (jthr) + return jthr; jmethodID mid = 0; - if (!validateMethodType(methType)) { - fprintf(stderr, "invalid method type\n"); - return NULL; - } - + jthr = validateMethodType(env, methType); + if (jthr) + return jthr; if (methType == STATIC) { mid = (*env)->GetStaticMethodID(env, cls, methName, methSignature); } @@ -344,72 +339,88 @@ jmethodID methodIdFromClass(const char *className, const char *methName, mid = (*env)->GetMethodID(env, cls, methName, methSignature); } if (mid == NULL) { - fprintf(stderr, "could not find method %s from class %s with signature %s\n",methName, className, methSignature); + fprintf(stderr, "could not find method %s from class %s with " + "signature %s\n", methName, className, methSignature); + return getPendingExceptionAndClear(env); } - return mid; + *out = mid; + return NULL; } - -jclass globalClassReference(const char *className, JNIEnv *env) +jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out) { jclass clsLocalRef; jclass cls = searchEntryFromTable(className); if (cls) { - return cls; + *out = cls; + return NULL; } - clsLocalRef = (*env)->FindClass(env,className); if (clsLocalRef == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; + return getPendingExceptionAndClear(env); } cls = (*env)->NewGlobalRef(env, clsLocalRef); if (cls == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; + (*env)->DeleteLocalRef(env, clsLocalRef); + return getPendingExceptionAndClear(env); } (*env)->DeleteLocalRef(env, clsLocalRef); insertEntryIntoTable(className, cls); - return cls; + *out = cls; + return NULL; } - -char *classNameOfObject(jobject jobj, JNIEnv *env) { - jclass cls, clsClass; +jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name) +{ + jthrowable jthr; + jclass cls, clsClass = NULL; jmethodID mid; - jstring str; - const char *cstr; + jstring str = NULL; + const char *cstr = NULL; char *newstr; cls = (*env)->GetObjectClass(env, jobj); if (cls == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; + jthr = getPendingExceptionAndClear(env); + goto done; } clsClass = (*env)->FindClass(env, "java/lang/Class"); if (clsClass == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; + jthr = getPendingExceptionAndClear(env); + goto done; } mid = (*env)->GetMethodID(env, clsClass, "getName", "()Ljava/lang/String;"); if (mid == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; + jthr = getPendingExceptionAndClear(env); + goto done; } str = (*env)->CallObjectMethod(env, cls, mid); if (str == NULL) { - (*env)->ExceptionDescribe(env); - return NULL; + jthr = getPendingExceptionAndClear(env); + goto done; } - cstr = (*env)->GetStringUTFChars(env, str, NULL); - newstr = strdup(cstr); - (*env)->ReleaseStringUTFChars(env, str, cstr); - if (newstr == NULL) { - perror("classNameOfObject: strdup"); - return NULL; + if (!cstr) { + jthr = getPendingExceptionAndClear(env); + goto done; } - return newstr; + newstr = strdup(cstr); + if (newstr == NULL) { + jthr = newRuntimeError(env, "classNameOfObject: out of memory"); + goto done; + } + *name = newstr; + jthr = NULL; + +done: + destroyLocalReference(env, cls); + destroyLocalReference(env, clsClass); + if (str) { + if (cstr) + (*env)->ReleaseStringUTFChars(env, str, cstr); + (*env)->DeleteLocalRef(env, str); + } + return jthr; } @@ -429,6 +440,7 @@ static JNIEnv* getGlobalJNIEnv(void) JNIEnv *env; jint rv = 0; jint noVMs = 0; + jthrowable jthr; rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), vmBufLength, &noVMs); if (rv != 0) { @@ -501,10 +513,11 @@ static JNIEnv* getGlobalJNIEnv(void) "with error: %d\n", rv); return NULL; } - if (invokeMethod(env, NULL, NULL, STATIC, NULL, + jthr = invokeMethod(env, NULL, STATIC, NULL, "org/apache/hadoop/fs/FileSystem", - "loadFileSystems", "()V") != 0) { - (*env)->ExceptionDescribe(env); + "loadFileSystems", "()V"); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, "loadFileSystems"); } } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h index 442eedfb1ae..f37dea739fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h @@ -37,14 +37,35 @@ typedef enum { INSTANCE } MethType; - -/** Used for returning an appropriate return value after invoking - * a method +/** + * Create a new malloc'ed C string from a Java string. + * + * @param env The JNI environment + * @param jstr The Java string + * @param out (out param) the malloc'ed C string + * + * @return NULL on success; the exception otherwise */ -typedef jvalue RetVal; +jthrowable newCStr(JNIEnv *env, jstring jstr, char **out); -/** Used for returning the exception after invoking a method */ -typedef jthrowable Exc; +/** + * Create a new Java string from a C string. + * + * @param env The JNI environment + * @param str The C string + * @param out (out param) the java string + * + * @return NULL on success; the exception otherwise + */ +jthrowable newJavaStr(JNIEnv *env, const char *str, jstring *out); + +/** + * 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. + */ +void destroyLocalReference(JNIEnv *env, jobject jObject); /** invokeMethod: Invoke a Static or Instance method. * className: Name of the class where the method can be found @@ -63,33 +84,27 @@ typedef jthrowable Exc; * RETURNS: -1 on error and 0 on success. If -1 is returned, exc will have a valid exception reference, and the result stored at retval is undefined. */ -int invokeMethod(JNIEnv *env, RetVal *retval, Exc *exc, MethType methType, +jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType, jobject instObj, const char *className, const char *methName, const char *methSignature, ...); -/** constructNewObjectOfClass: Invoke a constructor. - * className: Name of the class - * ctorSignature: the signature of the constructor "(arg-types)V" - * env: The JNIEnv pointer - * exc: If the ctor throws any exception, this will contain the reference - * Arguments to the ctor must be passed after ctorSignature - */ -jobject constructNewObjectOfClass(JNIEnv *env, Exc *exc, const char *className, +jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out, const char *className, const char *ctorSignature, ...); -jmethodID methodIdFromClass(const char *className, const char *methName, +jthrowable methodIdFromClass(const char *className, const char *methName, const char *methSignature, MethType methType, - JNIEnv *env); + JNIEnv *env, jmethodID *out); -jclass globalClassReference(const char *className, JNIEnv *env); +jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out); /** classNameOfObject: Get an object's class name. * @param jobj: The object. * @param env: The JNIEnv pointer. - * @return Returns a pointer to a string containing the class name. This string - * must be freed by the caller. + * @param name: (out param) On success, will contain a string containing the + * class name. This string must be freed by the caller. + * @return NULL on success, or the exception */ -char *classNameOfObject(jobject jobj, JNIEnv *env); +jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name); /** getJNIEnv: A helper function to get the JNIEnv* for the given thread. * If no JVM exists, then one will be created. JVM command line arguments @@ -99,8 +114,6 @@ char *classNameOfObject(jobject jobj, JNIEnv *env); * */ JNIEnv* getJNIEnv(void); -jarray constructNewArrayString(JNIEnv *env, Exc *exc, const char **elements, int size) ; - #endif /*LIBHDFS_JNI_HELPER_H*/ /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c index 7ffa148d5ac..a8156468966 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c @@ -16,6 +16,7 @@ * limitations under the License. */ +#include "exception.h" #include "jni_helper.h" #include "native_mini_dfs.h" @@ -41,6 +42,8 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf) jobject bld = NULL, bld2 = NULL, cobj = NULL; jvalue val; JNIEnv *env = getJNIEnv(); + jthrowable jthr; + if (!env) { fprintf(stderr, "nmdCreate: unable to construct JNIEnv.\n"); goto error; @@ -50,35 +53,38 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf) fprintf(stderr, "nmdCreate: OOM"); goto error; } - cobj = constructNewObjectOfClass(env, NULL, HADOOP_CONF, "()V"); - if (!cobj) { - fprintf(stderr, "nmdCreate: unable to construct Configuration\n"); + jthr = constructNewObjectOfClass(env, &cobj, HADOOP_CONF, "()V"); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "nmdCreate: new Configuration"); goto error_free_cl; } - bld = constructNewObjectOfClass(env, NULL, MINIDFS_CLUSTER_BUILDER, + jthr = constructNewObjectOfClass(env, &bld, MINIDFS_CLUSTER_BUILDER, "(L"HADOOP_CONF";)V", cobj); - if (!bld) { - fprintf(stderr, "nmdCreate: unable to construct " - "NativeMiniDfsCluster#Builder\n"); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "nmdCreate: NativeMiniDfsCluster#Builder#Builder"); goto error_dlr_cobj; } - if (invokeMethod(env, &val, NULL, INSTANCE, bld, - MINIDFS_CLUSTER_BUILDER, "format", - "(Z)L" MINIDFS_CLUSTER_BUILDER ";", conf->doFormat)) { - fprintf(stderr, "nmdCreate: failed to call Builder#doFormat\n"); + jthr = invokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER, + "format", "(Z)L" MINIDFS_CLUSTER_BUILDER ";", conf->doFormat); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, "nmdCreate: " + "Builder::format"); goto error_dlr_bld; } bld2 = val.l; - if (invokeMethod(env, &val, NULL, INSTANCE, bld, - MINIDFS_CLUSTER_BUILDER, "build", - "()L" MINIDFS_CLUSTER ";")) { - fprintf(stderr, "nmdCreate: failed to call Builder#build\n"); + jthr = invokeMethod(env, &val, INSTANCE, bld, MINIDFS_CLUSTER_BUILDER, + "build", "()L" MINIDFS_CLUSTER ";"); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "nmdCreate: Builder#build"); goto error_dlr_bld2; } cl->obj = (*env)->NewGlobalRef(env, val.l); if (!cl->obj) { - fprintf(stderr, "nmdCreate: failed to create global reference to " - "MiniDFSCluster\n"); + printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "nmdCreate: NewGlobalRef"); goto error_dlr_val; } (*env)->DeleteLocalRef(env, val.l); @@ -116,13 +122,17 @@ void nmdFree(struct NativeMiniDfsCluster* cl) int nmdShutdown(struct NativeMiniDfsCluster* cl) { JNIEnv *env = getJNIEnv(); + jthrowable jthr; + if (!env) { fprintf(stderr, "nmdShutdown: getJNIEnv failed\n"); return -EIO; } - if (invokeMethod(env, NULL, NULL, INSTANCE, cl->obj, - MINIDFS_CLUSTER, "shutdown", "()V")) { - fprintf(stderr, "nmdShutdown: MiniDFSCluster#shutdown failure\n"); + jthr = invokeMethod(env, NULL, INSTANCE, cl->obj, + MINIDFS_CLUSTER, "shutdown", "()V"); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "nmdShutdown: MiniDFSCluster#shutdown"); return -EIO; } return 0; @@ -130,15 +140,17 @@ int nmdShutdown(struct NativeMiniDfsCluster* cl) int nmdWaitClusterUp(struct NativeMiniDfsCluster *cl) { + jthrowable jthr; JNIEnv *env = getJNIEnv(); if (!env) { fprintf(stderr, "nmdWaitClusterUp: getJNIEnv failed\n"); return -EIO; } - if (invokeMethod(env, NULL, NULL, INSTANCE, cl->obj, - MINIDFS_CLUSTER, "waitClusterUp", "()V")) { - fprintf(stderr, "nmdWaitClusterUp: MiniDFSCluster#waitClusterUp " - "failure\n"); + jthr = invokeMethod(env, NULL, INSTANCE, cl->obj, + MINIDFS_CLUSTER, "waitClusterUp", "()V"); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "nmdWaitClusterUp: MiniDFSCluster#waitClusterUp "); return -EIO; } return 0; @@ -148,6 +160,7 @@ int nmdGetNameNodePort(struct NativeMiniDfsCluster *cl) { JNIEnv *env = getJNIEnv(); jvalue jVal; + jthrowable jthr; if (!env) { fprintf(stderr, "nmdHdfsConnect: getJNIEnv failed\n"); @@ -155,10 +168,11 @@ int nmdGetNameNodePort(struct NativeMiniDfsCluster *cl) } // Note: this will have to be updated when HA nativeMiniDfs clusters are // supported - if (invokeMethod(env, &jVal, NULL, INSTANCE, cl->obj, - MINIDFS_CLUSTER, "getNameNodePort", "()I")) { - fprintf(stderr, "nmdHdfsConnect: MiniDFSCluster#getNameNodePort " - "failure\n"); + jthr = invokeMethod(env, &jVal, INSTANCE, cl->obj, + MINIDFS_CLUSTER, "getNameNodePort", "()I"); + if (jthr) { + printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "nmdHdfsConnect: MiniDFSCluster#getNameNodePort"); return -EIO; } return jVal.i;