You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/11/09 00:06:07 UTC

svn commit: r472684 [2/2] - in /lucene/hadoop/trunk: ./ src/c++/libhdfs/ src/c++/libhdfs/tests/ src/c++/libhdfs/tests/conf/

Added: lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.c
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/hdfsJniHelper.c?view=auto&rev=472684
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.c (added)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.c Wed Nov  8 15:06:06 2006
@@ -0,0 +1,386 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string.h> 
+#include "hdfsJniHelper.h"
+
+static pthread_mutex_t hdfsHashMutex = PTHREAD_MUTEX_INITIALIZER;
+static volatile int hashTableInited = 0;
+
+#define LOCK_HASH_TABLE() pthread_mutex_lock(&hdfsHashMutex)
+#define UNLOCK_HASH_TABLE() pthread_mutex_unlock(&hdfsHashMutex)
+
+
+/** The Native return types that methods could return */
+#define VOID          'V'
+#define JOBJECT       'L'
+#define JARRAYOBJECT  '['
+#define JBOOLEAN      'Z'
+#define JBYTE         'B'
+#define JCHAR         'C'
+#define JSHORT        'S'
+#define JINT          'I'
+#define JLONG         'J'
+#define JFLOAT        'F'
+#define JDOUBLE       'D'
+
+
+/**
+ * Helpful macro to convert a pthread_t to a string
+ */
+#define GET_threadID(threadID, key, keySize) \
+    snprintf(key, keySize, "__hdfs_threadID__%u", (unsigned)(threadID)); 
+#define threadID_SIZE 32
+
+
+/**
+ * MAX_HASH_TABLE_ELEM: The maximum no. of entries in the hashtable.
+ * It's set to 4096 to account for (classNames + No. of threads)
+ */
+#define MAX_HASH_TABLE_ELEM 4096
+
+
+#define CHECK_EXCEPTION_IN_METH_INVOC \
+    if ((*env)->ExceptionCheck(env)) {\
+        (*env)->ExceptionDescribe(env);\
+        va_end(args);\
+        return -1;\
+    }\
+
+
+static void validateMethodType(MethType methType)
+{
+    if (methType != STATIC && methType != INSTANCE) {
+        fprintf(stderr, "Unimplemented method type\n");
+        exit(1);
+    }
+    return;
+}
+
+
+static void hashTableInit(void)
+{
+    if (!hashTableInited) {
+        LOCK_HASH_TABLE();
+        if (!hashTableInited) {
+            if (hcreate(MAX_HASH_TABLE_ELEM) == 0) {
+                fprintf(stderr, "error creating hashtable, <%d>: %s\n",
+                        errno, strerror(errno));
+                exit(1);
+            } 
+            hashTableInited = 1;
+        }
+        UNLOCK_HASH_TABLE();
+    }
+}
+
+
+static void insertEntryIntoTable(const char *key, void *data)
+{
+    ENTRY e, *ep;
+    if (key == NULL || data == NULL) {
+        return;
+    }
+    hashTableInit();
+    e.data = data;
+    e.key = (char*)key;
+    LOCK_HASH_TABLE();
+    ep = hsearch(e, ENTER);
+    UNLOCK_HASH_TABLE();
+    if (ep == NULL) {
+        fprintf(stderr, "error adding key (%s) to hash table, <%d>: %s\n",
+                key, errno, strerror(errno));
+        exit(1);
+    }  
+}
+
+
+
+static void* searchEntryFromTable(const char *key)
+{
+    ENTRY e,*ep;
+    if (key == NULL) {
+        return NULL;
+    }
+    hashTableInit();
+    e.key = (char*)key;
+    LOCK_HASH_TABLE();
+    ep = hsearch(e, FIND);
+    UNLOCK_HASH_TABLE();
+    if (ep != NULL) {
+        return ep->data;
+    }
+    return NULL;
+}
+
+
+
+int invokeMethod(JNIEnv *env, RetVal *retval, MethType methType,
+                 jobject instObj, const char *className,
+                 const char *methName, const char *methSignature, ...)
+{
+    va_list args;
+    jclass cls;
+    jmethodID mid;
+    const char *str; 
+    char returnType;
+    
+    validateMethodType(methType);
+    cls = globalClassReference(className, env);
+    mid = methodIdFromClass(className, methName, methSignature, 
+                            methType, env);
+    if (mid == NULL) {
+        (*env)->ExceptionDescribe(env);
+        exit(1);
+    }
+   
+    str = methSignature;
+    while (*str != ')') str++;
+    str++;
+    returnType = *str;
+    va_start(args, methSignature);
+    if (returnType == JOBJECT || returnType == JARRAYOBJECT) {
+        jobject jobj = NULL;
+        if (methType == STATIC) {
+            jobj = (*env)->CallStaticObjectMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            jobj = (*env)->CallObjectMethodV(env, instObj, mid, args);
+        }
+        CHECK_EXCEPTION_IN_METH_INVOC
+        retval->l = jobj;
+    }
+    else if (returnType == VOID) {
+        if (methType == STATIC) {
+            (*env)->CallStaticVoidMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            (*env)->CallVoidMethodV(env, instObj, mid, args);
+        }
+       CHECK_EXCEPTION_IN_METH_INVOC
+    }
+    else if (returnType == JBOOLEAN) {
+        jboolean jbool = 0;
+        if (methType == STATIC) {
+            jbool = (*env)->CallStaticBooleanMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            jbool = (*env)->CallBooleanMethodV(env, instObj, mid, args);
+        }
+        CHECK_EXCEPTION_IN_METH_INVOC
+        retval->z = jbool;
+    }
+    else if (returnType == JLONG) {
+        jlong jl = -1;
+        if (methType == STATIC) {
+            jl = (*env)->CallStaticLongMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            jl = (*env)->CallLongMethodV(env, instObj, mid, args);
+        }
+        CHECK_EXCEPTION_IN_METH_INVOC
+        retval->j = jl;
+    }
+    else if (returnType == JINT) {
+        jint ji = -1;
+        if (methType == STATIC) {
+            ji = (*env)->CallStaticIntMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            ji = (*env)->CallIntMethodV(env, instObj, mid, args);
+        }
+        CHECK_EXCEPTION_IN_METH_INVOC
+        retval->i = ji;
+    }
+    va_end(args);
+    return 0;
+}
+
+
+jobject constructNewObjectOfClass(JNIEnv *env, const char *className, 
+                                  const char *ctorSignature, ...)
+{
+    va_list args;
+    jclass cls;
+    jmethodID mid; 
+    jobject jobj;
+
+    cls = globalClassReference(className, env);
+    mid = methodIdFromClass(className, "<init>", ctorSignature, 
+                            INSTANCE, env);
+    if (mid == NULL) {
+        (*env)->ExceptionDescribe(env);
+        exit(1);
+    } 
+    va_start(args, ctorSignature);
+    jobj = (*env)->NewObjectV(env, cls, mid, args);
+    va_end(args);
+    if ((*env)->ExceptionCheck(env)) {
+        (*env)->ExceptionDescribe(env);
+    }
+    return jobj;
+}
+
+
+
+
+jmethodID methodIdFromClass(const char *className, const char *methName, 
+                            const char *methSignature, MethType methType, 
+                            JNIEnv *env)
+{
+    jclass cls = globalClassReference(className, env);
+    jmethodID mid = 0;
+    validateMethodType(methType);
+    if (methType == STATIC) {
+        mid = (*env)->GetStaticMethodID(env, cls, methName, methSignature);
+    }
+    else if (methType == INSTANCE) {
+        mid = (*env)->GetMethodID(env, cls, methName, methSignature);
+    }
+    return mid;
+}
+
+
+jclass globalClassReference(const char *className, JNIEnv *env)
+{
+    jclass clsLocalRef;
+    jclass cls = searchEntryFromTable(className);
+    if (cls) {
+        return cls; 
+    }
+
+    clsLocalRef = (*env)->FindClass(env,className);
+    if (clsLocalRef == NULL) {
+        (*env)->ExceptionDescribe(env);
+        exit(1);
+    }
+    cls = (*env)->NewGlobalRef(env, clsLocalRef);
+    if (cls == NULL) {
+        (*env)->ExceptionDescribe(env);
+        exit(1);
+    }
+    (*env)->DeleteLocalRef(env, clsLocalRef);
+    insertEntryIntoTable(className, cls);
+    return cls;
+}
+
+
+
+
+/**
+ * getJNIEnv: A helper function to get the JNIEnv* for the given thread.
+ * @param: None.
+ * @return The JNIEnv* corresponding to the thread.
+ */
+JNIEnv* getJNIEnv(void)
+{
+    char threadID[threadID_SIZE];
+
+    const jsize vmBufLength = 1;
+    JavaVM* vmBuf[vmBufLength]; 
+    JNIEnv *env;
+    jint rv = 0; 
+    jint noVMs = 0;
+
+    //Get the threadID and stringize it 
+    GET_threadID(pthread_self(), threadID, sizeof(threadID));
+
+    //See if you already have the JNIEnv* cached...
+    env = (JNIEnv*)searchEntryFromTable(threadID);
+    if (env != NULL) {
+        return env; 
+    }
+
+    //All right... some serious work required here!
+    //1. Initialize the HashTable
+    //2. LOCK!
+    //3. Check if any JVMs have been created here
+    //      Yes: Use it (we should only have 1 VM)
+    //      No: Create the JVM
+    //4. UNLOCK
+
+    hashTableInit();
+
+    LOCK_HASH_TABLE();
+
+    rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), vmBufLength, &noVMs);
+    if (rv != 0) {
+        fprintf(stderr, "JNI_GetCreatedJavaVMs failed with error: %d\n", rv);
+        exit(1);
+    }
+
+    if (noVMs == 0) {
+        //Get the environment variables for initializing the JVM
+        char *hadoopClassPath = getenv("CLASSPATH");
+        if (hadoopClassPath == NULL) {
+            fprintf(stderr, "Environment variable CLASSPATH not set!\n");
+            exit(-1);
+        } 
+        char *hadoopClassPathVMArg = "-Djava.class.path=";
+        size_t optHadoopClassPathLen = strlen(hadoopClassPath) + 
+          strlen(hadoopClassPathVMArg) + 1;
+        char *optHadoopClassPath = malloc(sizeof(char)*optHadoopClassPathLen);
+        snprintf(optHadoopClassPath, optHadoopClassPathLen,
+        	"%s%s", hadoopClassPathVMArg, hadoopClassPath);
+
+        //Create the VM
+        JavaVMInitArgs vm_args;
+        JavaVMOption options[1];
+        JavaVM *vm;
+        
+        // User classes
+        options[0].optionString = optHadoopClassPath;
+        // Print JNI-related messages      
+        //options[2].optionString = "-verbose:jni";
+
+        vm_args.version = JNI_VERSION_1_2;
+        vm_args.options = options;
+        vm_args.nOptions = 1; 
+        vm_args.ignoreUnrecognized = 1;
+
+        rv = JNI_CreateJavaVM(&vm, (void*)&env, &vm_args);
+        if (rv != 0) {
+            fprintf(stderr, "Call to JNI_CreateJavaVM failed "
+                    "with error: %d\n", rv);
+            exit(1);
+        }
+
+        free(optHadoopClassPath);
+    }
+    else {
+        //Attach this thread to the VM
+        JavaVM* vm = vmBuf[0];
+        rv = (*vm)->AttachCurrentThread(vm, (void*)&env, 0);
+        if (rv != 0) {
+            fprintf(stderr, "Call to AttachCurrentThread "
+                    "failed with error: %d\n", rv);
+            exit(1);
+        }
+    }
+
+    //Save the threadID -> env mapping
+    ENTRY e, *ep;
+    e.key = threadID;
+    e.data = (void*)(env);
+    if ((ep = hsearch(e, ENTER)) == NULL) {
+        fprintf(stderr, "Call to hsearch(ENTER) failed\n");
+        exit(1);
+    }
+
+    UNLOCK_HASH_TABLE();
+
+    return env;
+}

Modified: lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.h
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/hdfsJniHelper.h?view=diff&rev=472684&r1=472683&r2=472684
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.h (original)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.h Wed Nov  8 15:06:06 2006
@@ -26,46 +26,25 @@
 #include <stdarg.h>
 #include <search.h>
 #include <pthread.h>
-
-pthread_mutex_t hashTableMutex = PTHREAD_MUTEX_INITIALIZER; 
-
-#define LOCK_HASH_TABLE() pthread_mutex_lock(&hashTableMutex)
-#define UNLOCK_HASH_TABLE() pthread_mutex_unlock(&hashTableMutex)
+#include <errno.h>
 
 #define PATH_SEPARATOR ':'
 
 #define USER_CLASSPATH "/home/y/libexec/hadoop/conf:/home/y/libexec/hadoop/lib/hadoop-0.1.0.jar"
 
-/** The Native return types that methods could return */
-#define VOID          'V'
-#define JOBJECT       'L'
-#define JARRAYOBJECT  '['
-#define JBOOLEAN      'Z'
-#define JBYTE         'B'
-#define JCHAR         'C'
-#define JSHORT        'S'
-#define JINT          'I'
-#define JLONG         'J'
-#define JFLOAT        'F'
-#define JDOUBLE       'D'
 
 /** Denote the method we want to invoke as STATIC or INSTANCE */
-typedef enum { 
+typedef enum {
     STATIC,
     INSTANCE
 } MethType;
 
+
 /** Used for returning an appropriate return value after invoking
  * a method
  */
 typedef jvalue RetVal;
 
-/**
- * MAX_HASH_TABLE_ELEM: The maximum no. of entries in the hashtable.
- * It's set to 4096 to account for (classNames + No. of threads)
- */
-#define MAX_HASH_TABLE_ELEM 4096
-
 /** invokeMethod: Invoke a Static or Instance method.
  * className: Name of the class where the method can be found
  * methName: Name of the method
@@ -83,9 +62,8 @@
  * RETURNS: -1 on error and 0 on success. If -1 is returned, exc will have 
    a valid exception reference.
  */
-int invokeMethod(JNIEnv *env, RetVal *retval, jthrowable *exc,
-                 MethType methType, jobject instObj,
-                 const char *className, const char *methName, 
+int invokeMethod(JNIEnv *env, RetVal *retval, MethType methType,
+                 jobject instObj, const char *className, const char *methName, 
                  const char *methSignature, ...);
 
 /** constructNewObjectOfClass: Invoke a constructor.
@@ -95,239 +73,16 @@
  * 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, jthrowable *exc, 
-                                  const char *className, 
+jobject constructNewObjectOfClass(JNIEnv *env, const char *className, 
                                   const char *ctorSignature, ...);
 
-void validateMethodType(MethType methType);
-
 jmethodID methodIdFromClass(const char *className, const char *methName, 
                             const char *methSignature, MethType methType, 
                             JNIEnv *env);
 
 jclass globalClassReference(const char *className, JNIEnv *env);
 
-void insertEntryIntoTable(const char *key, void *data);
-void *searchEntryFromTable(const char *key);
-void hashTableInit();
-
-#define CHECK_EXCEPTION_IN_METH_INVOC {\
-    jthrowable _exc_;\
-    if((_exc_ = (*env)->ExceptionOccurred(env))) {\
-        (*env)->ExceptionDescribe(env);\
-        *exc = _exc_;\
-        (*env)->ExceptionClear(env);\
-        va_end(args);\
-        return -1;\
-    }\
-}
-
-int invokeMethod(JNIEnv *env, RetVal *retval, jthrowable *exc,
-                 MethType methType, jobject instObj,
-                 const char *className, const char *methName, 
-                 const char *methSignature, ...)
-{
-    va_list args;
-    jclass cls;
-    jmethodID mid;
-    const char *str; 
-    char returnType;
-    
-    validateMethodType(methType);
-    cls = globalClassReference(className, env);
-    mid = methodIdFromClass(className, methName, methSignature, 
-            methType, env);
-    if(mid == NULL) {
-        (*env)->ExceptionDescribe(env);
-        exit(1);
-    }
-   
-    str = methSignature;
-    while(*str != ')') str++;
-    str++;
-    returnType = *str;
-    va_start(args, methSignature);
-    if (returnType == JOBJECT || returnType == JARRAYOBJECT) {
-        jobject jobj;
-        if (methType == STATIC) {
-            jobj = (*env)->CallStaticObjectMethodV(env, cls, mid, args);
-        }
-        else if (methType == INSTANCE) {
-            jobj = (*env)->CallObjectMethodV(env, instObj, mid, args);
-        }
-        CHECK_EXCEPTION_IN_METH_INVOC
-        retval->l = jobj;
-    }
-    else if (returnType == VOID) {
-        if (methType == STATIC) {
-            (*env)->CallStaticVoidMethodV(env, cls, mid, args);
-        }
-        else if (methType == INSTANCE) {
-            (*env)->CallVoidMethodV(env, instObj, mid, args);
-        }
-        CHECK_EXCEPTION_IN_METH_INVOC
-    }
-    else if (returnType == JBOOLEAN) {
-        jboolean jbool;
-        if (methType == STATIC) {
-            jbool = (*env)->CallStaticBooleanMethodV(env, cls, mid, args);
-        }
-        else if (methType == INSTANCE) {
-            jbool = (*env)->CallBooleanMethodV(env, instObj, mid, args);
-        }
-        CHECK_EXCEPTION_IN_METH_INVOC
-        retval->z = jbool;
-    }
-    else if (returnType == JLONG) {
-        jlong jl;
-        if (methType == STATIC) {
-            jl = (*env)->CallStaticLongMethodV(env, cls, mid, args);
-        }
-        else if (methType == INSTANCE) {
-            jl = (*env)->CallLongMethodV(env, instObj, mid, args);
-        }
-        CHECK_EXCEPTION_IN_METH_INVOC
-        retval->j = jl;
-    }
-    else if (returnType == JINT) {
-        jint ji;
-        if (methType == STATIC) {
-            ji = (*env)->CallStaticIntMethodV(env, cls, mid, args);
-        }
-        else if (methType == INSTANCE) {
-            ji = (*env)->CallIntMethodV(env, instObj, mid, args);
-        }
-        CHECK_EXCEPTION_IN_METH_INVOC
-        retval->i = ji;
-    }
-    va_end(args);
-    return 0;
-}
-
-void validateMethodType(MethType methType)
-{
-    if (methType != STATIC && methType != INSTANCE) {
-        fprintf(stderr,"Unimplemented method type\n");
-        exit(1);
-    }
-    return;
-}
-
-jobject constructNewObjectOfClass(JNIEnv *env, jthrowable *exc, 
-                                  const char *className, 
-                                  const char *ctorSignature, ...)
-{
-    va_list args;
-    jclass cls;
-    jmethodID mid; 
-    jobject jobj;
-    jthrowable _exc;
-
-    cls = globalClassReference(className, env);
-    mid = methodIdFromClass(className, "<init>", ctorSignature, 
-            INSTANCE, env);
-    if(mid == NULL) {
-        (*env)->ExceptionDescribe(env);
-        exit(1);
-    } 
-    va_start(args, ctorSignature);
-    jobj = (*env)->NewObjectV(env, cls, mid, args);
-    va_end(args);
-    if((_exc = (*env)->ExceptionOccurred(env))) {
-        (*env)->ExceptionDescribe(env);
-        *exc = _exc;
-        (*env)->ExceptionClear(env);
-    }
-    return jobj;
-}
-
-jmethodID methodIdFromClass(const char *className, const char *methName, 
-                            const char *methSignature, MethType methType, 
-                            JNIEnv *env)
-{
-    jclass cls = globalClassReference(className, env);
-    jmethodID mid;
-    validateMethodType(methType);
-    if(methType == STATIC) {
-        mid = (*env)->GetStaticMethodID(env, cls, methName, methSignature);
-    }
-    else if(methType == INSTANCE) {
-        mid = (*env)->GetMethodID(env, cls, methName, methSignature);
-    }
-    return mid;
-}
-
-jclass globalClassReference(const char *className, JNIEnv *env)
-{
-    jclass clsLocalRef;
-    jclass cls = searchEntryFromTable(className);
-    if(cls) {
-        return cls; 
-    }
-
-    clsLocalRef = (*env)->FindClass(env,className);
-    if(clsLocalRef == NULL) {
-        (*env)->ExceptionDescribe(env);
-        exit(1);
-    }
-    cls = (*env)->NewGlobalRef(env, clsLocalRef);
-    if(cls == NULL) {
-        (*env)->ExceptionDescribe(env);
-        exit(1);
-    }
-    (*env)->DeleteLocalRef(env, clsLocalRef);
-    insertEntryIntoTable(className, cls);
-    return cls;
-}
-
-void insertEntryIntoTable(const char *key, void *data)
-{
-    ENTRY e, *ep;
-    if(key == NULL || data == NULL) {
-        return;
-    }
-    hashTableInit();
-    e.data = data;
-    e.key = (char*)key;
-    LOCK_HASH_TABLE();
-    ep = hsearch(e, ENTER);
-    UNLOCK_HASH_TABLE();
-    if(ep == NULL) {
-        fprintf(stderr,"hsearch(ENTER) returned error\n");
-        exit(1);
-    }  
-}
-
-void *searchEntryFromTable(const char *key)
-{
-    ENTRY e,*ep;
-    if(key == NULL) {
-        return NULL;
-    }
-    hashTableInit();
-    e.key = (char*)key;
-    LOCK_HASH_TABLE();
-    ep = hsearch(e, FIND);
-    UNLOCK_HASH_TABLE();
-    if(ep != NULL) {
-        return ep->data;
-    }
-    return NULL;
-}
-
-void hashTableInit()
-{
-    static int hash_table_inited = 0;
-    LOCK_HASH_TABLE();
-    if(!hash_table_inited) {
-        if (hcreate(MAX_HASH_TABLE_ELEM) == 0) {
-            fprintf(stderr,"hcreate returned error\n");
-            exit(1);
-        } 
-        hash_table_inited = 1;
-    }  
-    UNLOCK_HASH_TABLE();
-}
+JNIEnv* getJNIEnv(void);
 
 #endif /*LIBHDFS_JNI_HELPER_H*/
 

Modified: lucene/hadoop/trunk/src/c++/libhdfs/hdfs_test.c
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/hdfs_test.c?view=diff&rev=472684&r1=472683&r2=472684
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfs_test.c (original)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfs_test.c Wed Nov  8 15:06:06 2006
@@ -70,6 +70,13 @@
         //Read tests
         
         const char* readPath = "/tmp/testfile.txt";
+        int exists = hdfsExists(fs, readPath);
+
+        if (exists) {
+          fprintf(stderr, "Failed to validate existence of %s\n", readPath);
+          exit(-1);
+        }
+
         hdfsFile readFile = hdfsOpenFile(fs, readPath, O_RDONLY, 0, 0, 0);
         if (!readFile) {
             fprintf(stderr, "Failed to open %s for reading!\n", readPath);
@@ -188,6 +195,7 @@
         fprintf(stderr, "hdfsDelete: %s\n", (hdfsDelete(fs, srcPath) ? "Failed!" : "Success!"));
         fprintf(stderr, "hdfsDelete: %s\n", (hdfsDelete(lfs, srcPath) ? "Failed!" : "Success!"));
         fprintf(stderr, "hdfsDelete: %s\n", (hdfsDelete(lfs, dstPath) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsExists: %s\n", (hdfsExists(fs, newDirectory) ? "Success!" : "Failed!"));
     }
 
     return 0;

Modified: lucene/hadoop/trunk/src/c++/libhdfs/tests/conf/hadoop-site.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/tests/conf/hadoop-site.xml?view=diff&rev=472684&r1=472683&r2=472684
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/tests/conf/hadoop-site.xml (original)
+++ lucene/hadoop/trunk/src/c++/libhdfs/tests/conf/hadoop-site.xml Wed Nov  8 15:06:06 2006
@@ -8,28 +8,17 @@
 <configuration>
 
 <property>
-  <name>fs.default.name</name>
-  <value>localhost:23000</value>
-  <description>The name of the default file system.  Either the
-  literal string "local" or a host:port for DFS.</description>
+  <name>hadoop.tmp.dir</name>
+  <value>../../../build/test/libhdfs</value>
+  <description>A base for other temporary directories.</description>
 </property>
 
-<property>
-  <name>dfs.name.dir</name>
-  <value>/tmp/test-libhdfs/dfs/name</value>
-  <description>Determines where on the local filesystem the DFS name node
-      should store the name table.</description>
-</property>
 
 <property>
-  <name>dfs.data.dir</name>
-  <value>/tmp/test-libhdfs/dfs/data</value>
-  <description>Determines where on the local filesystem an DFS data node
-  should store its blocks.  If this is a comma-delimited
-  list of directories, then data will be stored in all named
-  directories, typically on different devices.
-  Directories that do not exist are ignored.
-  </description>
+  <name>fs.default.name</name>
+  <value>localhost:23000</value>
+  <description>The name of the default file system.  Either the
+  literal string "local" or a host:port for DFS.</description>
 </property>
 
 <property>

Modified: lucene/hadoop/trunk/src/c++/libhdfs/tests/test-libhdfs.sh
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/tests/test-libhdfs.sh?view=diff&rev=472684&r1=472683&r2=472684
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/tests/test-libhdfs.sh (original)
+++ lucene/hadoop/trunk/src/c++/libhdfs/tests/test-libhdfs.sh Wed Nov  8 15:06:06 2006
@@ -28,13 +28,40 @@
 HADOOP_LIB_DIR=$HADOOP_HOME/lib
 HADOOP_BIN_DIR=$HADOOP_HOME/bin
 
-# Add libs to CLASSPATH for libhdfs (jni)
-CLASSPATH=`for f in $HADOOP_LIB_DIR/*.jar; do CLASSPATH=$CLASSPATH:$f; done; echo $CLASSPATH;`
-
-# Manipulate HADOOP_CONF_DIR so as to include 
-# HADOOP_HOME/conf/hadoop-default.xml too
+## Manipulate HADOOP_CONF_DIR so as to include 
+## HADOOP_HOME/conf/hadoop-default.xml too
 # which is necessary to circumvent bin/hadoop
 HADOOP_CONF_DIR=$HADOOP_CONF_DIR:$HADOOP_HOME/conf
+
+# CLASSPATH initially contains $HADOOP_CONF_DIR
+CLASSPATH="${HADOOP_CONF_DIR}"
+CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar
+
+# for developers, add Hadoop classes to CLASSPATH
+if [ -d "$HADOOP_HOME/build/classes" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
+fi
+if [ -d "$HADOOP_HOME/build/webapps" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
+fi
+if [ -d "$HADOOP_HOME/build/test/classes" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/test/classes
+fi
+
+# so that filenames w/ spaces are handled correctly in loops below
+IFS=
+
+# add libs to CLASSPATH
+for f in $HADOOP_HOME/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+for f in $HADOOP_HOME/lib/jsp-2.0/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+# restore ordinary behaviour
+unset IFS
 
 # Put delays to ensure hdfs is up and running and also shuts down 
 # after the tests are complete