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/05/31 21:08:39 UTC

svn commit: r410637 [1/2] - in /lucene/hadoop/trunk: ./ src/c++/libhdfs/ src/test/org/apache/hadoop/fs/ src/test/org/apache/hadoop/test/

Author: cutting
Date: Wed May 31 12:08:38 2006
New Revision: 410637

URL: http://svn.apache.org/viewvc?rev=410637&view=rev
Log:
Add a C API to DFS.  Contributed by Arun C Murthy.

Added:
    lucene/hadoop/trunk/src/c++/libhdfs/
    lucene/hadoop/trunk/src/c++/libhdfs/Makefile
    lucene/hadoop/trunk/src/c++/libhdfs/hdfs.c
    lucene/hadoop/trunk/src/c++/libhdfs/hdfs.h
    lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.h
    lucene/hadoop/trunk/src/c++/libhdfs/hdfs_read.c
    lucene/hadoop/trunk/src/c++/libhdfs/hdfs_test.c
    lucene/hadoop/trunk/src/c++/libhdfs/hdfs_write.c
    lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestDFSCIO.java
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/build.xml
    lucene/hadoop/trunk/src/test/org/apache/hadoop/test/AllTestDriver.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=410637&r1=410636&r2=410637&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed May 31 12:08:38 2006
@@ -90,6 +90,8 @@
     of filesystem integrity problems, when, e.g., datanodes are
     restarted.  (Konstantin Shvachko via cutting)
 
+24. HADOOP-256.  Add a C API for DFS.  (Arun C Murthy via cutting)
+
 
 Release 0.2.1 - 2006-05-12
 

Modified: lucene/hadoop/trunk/build.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/build.xml?rev=410637&r1=410636&r2=410637&view=diff
==============================================================================
--- lucene/hadoop/trunk/build.xml (original)
+++ lucene/hadoop/trunk/build.xml Wed May 31 12:08:38 2006
@@ -12,6 +12,7 @@
   <property name="version" value="0.3-dev"/>
   <property name="final.name" value="${name}-${version}"/>
   <property name="year" value="2006"/>
+  <property name="libhdfs.version" value="1"/>
 
   <property name="src.dir" value="${basedir}/src/java"/>
   <property name="examples.dir" value="${basedir}/src/examples"/>
@@ -25,6 +26,7 @@
   <property name="build.src" value="${build.dir}/src"/>
   <property name="build.webapps" value="${build.dir}/webapps"/>
   <property name="build.examples" value="${build.dir}/examples"/>
+  <property name="build.libhdfs" value="${build.dir}/libhdfs"/>
   <property name="build.docs" value="${build.dir}/docs"/>
   <property name="build.javadoc" value="${build.docs}/api"/>
   <property name="build.encoding" value="ISO-8859-1"/>
@@ -84,6 +86,7 @@
     <mkdir dir="${build.webapps}/task/WEB-INF"/>
     <mkdir dir="${build.webapps}/job/WEB-INF"/>
     <mkdir dir="${build.examples}"/>
+    <mkdir dir="${build.libhdfs}"/>
 
     <mkdir dir="${test.build.dir}"/>
     <mkdir dir="${test.build.classes}"/>
@@ -120,7 +123,7 @@
       />
   </target>
 
-  <target name="compile" depends="init, record-parser">
+  <target name="compile" depends="init, record-parser, compile-libhdfs">
 
     <jsp-compile
      uriroot="${src.webapps}/task"
@@ -312,10 +315,11 @@
   <!-- ================================================================== -->
   <!--                                                                    -->
   <!-- ================================================================== -->
-  <target name="package" depends="jar, javadoc, examples, compile-test">
+  <target name="package" depends="jar, javadoc, examples, compile-test, compile-libhdfs">
     <mkdir dir="${dist.dir}"/>
     <mkdir dir="${dist.dir}/lib"/>
     <mkdir dir="${dist.dir}/bin"/>
+    <mkdir dir="${dist.dir}/libhdfs"/>
     <mkdir dir="${dist.dir}/docs"/>
     <mkdir dir="${dist.dir}/docs/api"/>
 
@@ -345,6 +349,18 @@
         <fileset dir="${dist.dir}/bin"/>
     </chmod>
 
+    <copy todir="${dist.dir}/libhdfs">
+      <fileset dir="${build.libhdfs}" followsymlinks="false"/>
+    </copy>
+
+    <chmod perm="ugo+x" type="file">
+        <fileset dir="${dist.dir}/libhdfs"/>
+    </chmod>
+
+    <exec dir="${dist.dir}/libhdfs" executable="ln">
+		<arg line="-sf libhdfs.so.${libhdfs.version} libhdfs.so"/>
+     </exec>
+
     <copy todir="${dist.dir}/docs">
       <fileset dir="${build.docs}"/>
     </copy>
@@ -406,4 +422,26 @@
      </subant>  	
   </target>
 
+  <!-- ================================================================== -->
+  <!-- libhdfs targets. For now, they must be called explicitly.          -->
+  <!-- ================================================================== -->
+  <target name="compile-libhdfs" depends="init">
+    <exec dir="${basedir}/src/c++/libhdfs" executable="make">
+      <env key="OS_NAME" value="${os.name}"/>
+      <env key="OS_ARCH" value="${os.arch}"/>
+      <env key="SHLIB_VERSION" value="${libhdfs.version}"/>
+      <env key="LIBHDFS_BUILD_DIR" value="${build.libhdfs}"/>
+    </exec>
+  </target>
+  <target name="test-libhdfs" depends="compile-libhdfs">
+    <exec dir="${build.libhdfs}" executable="${build.libhdfs}/hdfs_test">
+      <env key="LD_PRELOAD" value="./libhdfs.so"/>
+    </exec>
+  </target>
+  <target name="clean-libhdfs">
+    <exec dir="${basedir}/src/c++/libhdfs" executable="make">
+    		<arg value="clean"/>
+    </exec>
+  </target>
+		
 </project>

Added: lucene/hadoop/trunk/src/c++/libhdfs/Makefile
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/Makefile?rev=410637&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/Makefile (added)
+++ lucene/hadoop/trunk/src/c++/libhdfs/Makefile Wed May 31 12:08:38 2006
@@ -0,0 +1,61 @@
+#
+# Copyright 2005 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.
+#
+
+#
+# Note: This makefile depends on 3 environment variables to funtion correctly:
+# a) JAVA_HOME
+# b) OS_NAME
+# c) OS_ARCH
+# d) LIBHDFS_BUILD_DIR
+# All these are passed by build.xml.
+#
+
+CC = gcc
+LD = gcc
+CFLAGS =  -m32 -g -W -fPIC
+LDFLAGS = -L$(JAVA_HOME)/jre/lib/$(OS_ARCH)/server -ljvm -m32 -shared -Wl,-x 
+PLATFORM = `echo $$OS_NAME | /usr/bin/tr [A-Z] [a-z]`
+LINC = -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/$(PLATFORM)
+HDFS_TESTS = hdfs_test
+HDFS_READ_TEST = hdfs_read
+HDFS_WRITE_TEST = hdfs_write
+LIB_NAME = hdfs
+OBJS = hdfs.o
+RM = /bin/rm -f
+LINK = ln -sf
+
+all: $(LIB_NAME) $(HDFS_TESTS) $(HDFS_READ_TEST) $(HDFS_WRITE_TEST) 
+
+$(LIB_NAME): $(OBJS)
+	$(LD) $(LDFLAGS) -o $(LIBHDFS_BUILD_DIR)/lib$(LIB_NAME).so.$(SHLIB_VERSION) -Wl,-soname,lib$(LIB_NAME).so.$(SHLIB_VERSION) $(OBJS) \
+	&& $(LINK) $(LIBHDFS_BUILD_DIR)/lib$(LIB_NAME).so.$(SHLIB_VERSION) $(LIBHDFS_BUILD_DIR)/lib$(LIB_NAME).so
+
+$(OBJS): hdfs.c
+	$(CC) $(CFLAGS) $(LINC) -c hdfs.c
+
+$(HDFS_TESTS): hdfs_test.c
+	$(CC) $(HDFS_TESTS).c $(LINC) -L$(LIBHDFS_BUILD_DIR) -l$(LIB_NAME) -m32 -o $(LIBHDFS_BUILD_DIR)/$(HDFS_TESTS)
+
+$(HDFS_READ_TEST): hdfs_read.c
+	$(CC) $(HDFS_READ_TEST).c $(LINC) -m32 -Wl,-rpath,. -L$(LIBHDFS_BUILD_DIR) -l$(LIB_NAME) -o $(LIBHDFS_BUILD_DIR)/$(HDFS_READ_TEST)
+
+$(HDFS_WRITE_TEST): hdfs_write.c
+	$(CC) $(HDFS_WRITE_TEST).c $(LINC) -m32 -Wl,-rpath,. -L$(LIBHDFS_BUILD_DIR) -l$(LIB_NAME) -o $(LIBHDFS_BUILD_DIR)/$(HDFS_WRITE_TEST)
+
+clean:
+	$(RM) $(LIBHDFS_BUILD_DIR)/*.o $(LIBHDFS_BUILD_DIR)/*.so* \
+	$(LIBHDFS_BUILD_DIR)/$(HDFS_TESTS) $(LIBHDFS_BUILD_DIR)/$(HDFS_READ_TEST) $(LIBHDFS_BUILD_DIR)/$(HDFS_WRITE_TEST)
+

Added: lucene/hadoop/trunk/src/c++/libhdfs/hdfs.c
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/hdfs.c?rev=410637&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfs.c (added)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfs.c Wed May 31 12:08:38 2006
@@ -0,0 +1,1513 @@
+/**
+ * Copyright 2005 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 "hdfs.h"
+#include "hdfsJniHelper.h"
+
+/**
+ * hdfsJniEnv: A wrapper struct to be used as 'value'
+ * while saving thread -> JNIEnv* mappings
+ */
+typedef struct 
+{
+    JNIEnv* env;
+} hdfsJniEnv;
+
+/**
+ * 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
+
+#define CHECK_jExceptionEPTION_IN_METH_INVOC {\
+    jthrowable _jException_;\
+    if ((_jException_ = (*env)->jExceptioneptionOccurred(env))) {\
+        (*env)->jExceptioneptionDescribe(env);\
+        *jException = _jException_;\
+        (*env)->jExceptioneptionClear(env);\
+        va_end(args);\
+        return -1;\
+    }\
+}
+
+/**
+ * getJNIEnv: A helper function to get the JNIEnv* for the given thread.
+ * @param: None.
+ * @return The JNIEnv* corresponding to the thread.
+ */
+static inline JNIEnv* getJNIEnv()
+{
+    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,
+                "Call to 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, "Please set the environment variable $CLASSPATH!\n");
+        		exit(-1);
+        } 
+        char *optHadoopClassPath = malloc(sizeof(char) * strlen(hadoopClassPath) + 16);
+        sprintf(optHadoopClassPath, "-Djava.class.path=%s", 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");
+            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");
+            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;
+}
+
+/**
+ * Helper function to create a java.io.File object.
+ * @param env: The JNIEnv pointer. 
+ * @param path: The file-path for which to construct java.io.File object.
+ * @return Returns a jobject on success and NULL on error.
+ */
+static inline jobject constructNewObjectOfJavaIOFile(JNIEnv *env, const char *path)
+{
+    //Construct a java.lang.String object
+    jstring jPath = (*env)->NewStringUTF(env, path); 
+
+    //Construct the java.io.File object
+    jthrowable jException;
+    jobject jFile = constructNewObjectOfClass(env, &jException, 
+            "java/io/File", "(Ljava/lang/String;)V", jPath);
+    if (jFile == NULL) {
+        fprintf(stderr, 
+                "Can't construct instance of class java.io.File for %s\n",
+                path);
+        errno = EINTERNAL;
+        return NULL;
+    }
+
+    //Destroy the java.lang.String object
+    (*env)->ReleaseStringUTFChars(env, jPath,
+                (*env)->GetStringUTFChars(env, jPath, 0));
+
+    return jFile;
+}
+
+/**
+ * Helper function to create a org.apache.hadoop.fs.Path object.
+ * @param env: The JNIEnv pointer. 
+ * @param path: The file-path for which to construct org.apache.hadoop.fs.Path object.
+ * @return Returns a jobject on success and NULL on error.
+ */
+static inline 
+jobject constructNewObjectOfPath(JNIEnv *env, const char *path)
+{
+    //Construct a java.lang.String object
+    jstring jPathString = (*env)->NewStringUTF(env, path); 
+
+    //Construct the org.apache.hadoop.fs.Path object
+    jthrowable jException;
+    jobject jPath = constructNewObjectOfClass(env, &jException, 
+            "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 java.lang.String object
+    (*env)->ReleaseStringUTFChars(env, jPathString,
+                (*env)->GetStringUTFChars(env, jPathString, 0));
+
+    return jPath;
+}
+
+/**
+ * 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 inline void destroyLocalReference(JNIEnv *env, jobject jObject)
+{
+  (*env)->DeleteLocalRef(env, jObject);
+}
+
+hdfsFS hdfsConnect(const char* host, tPort port)
+{
+    // JAVA EQUIVALENT:
+    //  FileSystem fs = FileSystem.get(new Configuration());
+    //  return fs;
+
+    JNIEnv *env = 0;
+    jobject jConfiguration;
+    jobject jFS;
+    jthrowable jException;
+
+    //Get the JNIEnv* corresponding to current thread
+    env = getJNIEnv();
+
+    //Create the org.apache.hadoop.conf.Configuration object
+    jConfiguration = constructNewObjectOfClass(env, &jException, 
+            "org/apache/hadoop/conf/Configuration", "()V");
+    if (jConfiguration == NULL) {
+        fprintf(stderr,
+                "Can't construct instance of class org.apache.hadoop.conf.Configuration\n");
+        errno = EINTERNAL;
+        return NULL;
+    }
+ 
+    //Check what type of FileSystem the caller wants...
+    if (host == NULL) {
+        //fs = new LocalFileSystem(conf);
+        jFS = constructNewObjectOfClass(env, &jException,
+                "org/apache/hadoop/fs/LocalFileSystem",
+                "(Lorg/apache/hadoop/conf/Configuration;)V", jConfiguration);
+        if (jFS == NULL) {
+            errno = EINTERNAL;
+            goto done;
+        }
+    } else if (!strcmp(host, "default") && port == 0) {
+        //fs = FileSystem::get(conf); 
+        if (invokeMethod(env, (RetVal*)&jFS, &jException, STATIC, NULL,
+                    "org/apache/hadoop/fs/FileSystem", "get", 
+                    "(Lorg/apache/hadoop/conf/Configuration;)Lorg/apache/hadoop/fs/FileSystem;", 
+                    jConfiguration) != 0) {
+            fprintf(stderr, 
+                    "Call to org.apache.hadoop.fs.FileSystem::get failed!\n");
+            errno = EINTERNAL;
+            goto done;
+        }
+    } else {
+        //fs = new DistributedFileSystem(new InetSocketAddress(host, port), conf)
+        jstring jHostName = (*env)->NewStringUTF(env, host);
+    
+        jobject jNameNode = constructNewObjectOfClass(env, &jException,
+                "java/net/InetSocketAddress", "(Ljava/lang/String;I)V", 
+                jHostName, port);
+        (*env)->ReleaseStringUTFChars(env, jHostName,
+                            (*env)->GetStringUTFChars(env, jHostName, NULL));
+        if (jNameNode == NULL) {
+            errno = EINTERNAL;
+            goto done;
+        }
+    
+        jFS = constructNewObjectOfClass(env, &jException,
+                "org/apache/hadoop/dfs/DistributedFileSystem",
+                "(Ljava/net/InetSocketAddress;Lorg/apache/hadoop/conf/Configuration;)V", 
+                jNameNode, jConfiguration);
+        destroyLocalReference(env, jNameNode);
+        if (jFS == NULL) {
+            errno = EINTERNAL;
+            goto done;
+        }
+    }
+
+    done:
+    
+    //Release unnecessary local references
+    destroyLocalReference(env, jConfiguration);
+
+    return jFS;
+}
+
+int hdfsDisconnect(hdfsFS fs)
+{
+    // JAVA EQUIVALENT:
+    //  fs.close()
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+
+    //jException reference
+    jthrowable jException;
+
+    //Sanity check
+    if (fs == NULL) {
+        errno = EBADF;
+        return -1;
+    }
+
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem",
+                "close", "()V") != 0) {
+        fprintf(stderr, "Call to FileSystem::close failed!\n"); 
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    //Release unnecessary local references
+    destroyLocalReference(env, jFS);
+
+    return 0;
+}
+
+hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags, 
+        int bufferSize, short replication, tSize blockSize)
+{
+    // JAVA EQUIVALENT:
+    //  File f = new File(path);
+    //  FSData{Input|Output}Stream f{is|os} = fs.create(f);
+    //  return f{is|os};
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //The hadoop java api/signature
+    const char* method = (flags == O_RDONLY) ? "open" : "create";
+    const char* signature = (flags == O_RDONLY) ? 
+        "(Lorg/apache/hadoop/fs/Path;I)Lorg/apache/hadoop/fs/FSDataInputStream;" : 
+        "(Lorg/apache/hadoop/fs/Path;ZISJ)Lorg/apache/hadoop/fs/FSDataOutputStream;";
+
+    //Return value
+    hdfsFile file = NULL;
+
+    //Create an object of org.apache.hadoop.fs.Path 
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return NULL; 
+    }
+
+    //Create the org.apache.hadoop.conf.Configuration object
+    //and get the configured values if need be
+    jobject jConfiguration = constructNewObjectOfClass(env, &jException, 
+            "org/apache/hadoop/conf/Configuration", "()V");
+    if (jConfiguration == NULL) {
+        fprintf(stderr,
+                "Can't construct instance of class org.apache.hadoop.conf.Configuration\n");
+        errno = EINTERNAL;
+        return NULL;
+    }
+    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"); 
+
+    //bufferSize
+    if(!bufferSize) {
+        if (invokeMethod(env, (RetVal*)&jBufferSize, &jException, INSTANCE, jConfiguration, 
+                    "org/apache/hadoop/conf/Configuration", "getInt",
+                    "(Ljava/lang/String;I)I", jStrBufferSize, 4096)) {
+            fprintf(stderr,
+                    "Call to org.apache.hadoop.conf.Configuration::getInt failed!\n");
+            errno = EINTERNAL;
+            goto done;
+        }
+    }
+
+    if(flags == O_WRONLY) {
+        //replication
+        jint jTmpReplication;
+        if(!replication) {
+            if (invokeMethod(env, (RetVal*)&jTmpReplication, &jException, INSTANCE, jConfiguration, 
+                        "org/apache/hadoop/conf/Configuration", "getInt",
+                        "(Ljava/lang/String;I)I", jStrReplication, 1)) {
+                fprintf(stderr,
+                        "Call to org.apache.hadoop.conf.Configuration::getInt failed!\n");
+                errno = EINTERNAL;
+                goto done;
+            }
+            jReplication = jTmpReplication;
+        }
+        
+        //blockSize
+        if(!blockSize) {
+            if (invokeMethod(env, (RetVal*)&jBlockSize, &jException, INSTANCE, jConfiguration, 
+                        "org/apache/hadoop/conf/Configuration", "getLong",
+                        "(Ljava/lang/String;J)J", jStrBlockSize, 67108864)) {
+                fprintf(stderr,
+                        "Call to org.apache.hadoop.fs.FileSystem::%s(%s) failed!\n", 
+                        method, signature);
+                errno = EINTERNAL;
+                goto done;
+            }
+        }
+    }
+ 
+    //Create and return either the FSDataInputStream or FSDataOutputStream references 
+    jobject jStream;
+    if(flags == O_RDONLY) {
+        if (invokeMethod(env, (RetVal*)&jStream, &jException, INSTANCE, jFS, 
+                    "org/apache/hadoop/fs/FileSystem", 
+                    method, signature, jPath, jBufferSize)) {
+            fprintf(stderr,
+                    "Call to org.apache.hadoop.fs.FileSystem::%s(%s) failed!\n", 
+                    method, signature);
+            errno = EINTERNAL;
+            goto done;
+        }
+    } else {
+        jboolean jOverWrite = 1;
+        if (invokeMethod(env, (RetVal*)&jStream, &jException, INSTANCE, jFS, 
+                    "org/apache/hadoop/fs/FileSystem", 
+                    method, signature, jPath, jOverWrite, jBufferSize, jReplication, jBlockSize)) {
+            fprintf(stderr,
+                    "Call to org.apache.hadoop.fs.FileSystem::%s(%s) failed!\n", 
+                    method, signature);
+            errno = EINTERNAL;
+            goto done;
+        }
+    }
+  
+    file = malloc(sizeof(struct hdfsFile_internal));
+    if (!file) {
+        errno = ENOMEM;
+        return NULL;
+    }
+    file->file = (void*)jStream;
+    file->type = ((flags & O_RDONLY) ? INPUT : OUTPUT);
+
+    done:
+
+    //Delete unnecessary local references
+    (*env)->ReleaseStringUTFChars(env, jStrBufferSize,
+                (*env)->GetStringUTFChars(env, jStrBufferSize, 0));
+    (*env)->ReleaseStringUTFChars(env, jStrReplication,
+                (*env)->GetStringUTFChars(env, jStrReplication, 0));
+    (*env)->ReleaseStringUTFChars(env, jStrBlockSize,
+                (*env)->GetStringUTFChars(env, jStrBlockSize, 0));
+    destroyLocalReference(env, jConfiguration); 
+    destroyLocalReference(env, jPath); 
+
+    return file;
+}
+
+int hdfsCloseFile(hdfsFS fs, hdfsFile file)
+{
+    // JAVA EQUIVALENT:
+    //  file.close 
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jobject jStream = (jobject)(file ? file->file : NULL);
+
+    //jException reference
+    jthrowable jException;
+
+    //Sanity check
+    if (!file || file->type == UNINITIALIZED) {
+        errno = EBADF;
+        return -1;
+    }
+
+    //The interface whose 'close' method to be called
+    const char* interface = (file->type == INPUT) ? 
+        "org/apache/hadoop/fs/FSDataInputStream" : 
+        "org/apache/hadoop/fs/FSDataOutputStream";
+  
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jStream, interface,
+                "close", "()V") != 0) {
+        fprintf(stderr, "Call to %s::close failed!\n", interface); 
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    //De-allocate memory
+    free(file);
+
+    return 0;
+}
+
+tSize hdfsRead(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
+{
+    // JAVA EQUIVALENT:
+    //  byte [] bR = new byte[length];
+    //  fis.read(bR);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jobject jInputStream = (jobject)(f ? f->file : NULL);
+
+    jthrowable jException;
+    jbyteArray jbRarray;
+    jint noReadBytes = 0;
+
+    //Sanity check
+    if (!f || f->type == UNINITIALIZED) {
+        errno = EBADF;
+        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");
+        errno = EINVAL;
+        return -1;
+    }
+
+    //Read the requisite bytes
+    jbRarray = (*env)->NewByteArray(env, length);
+    if (invokeMethod(env, (RetVal*)&noReadBytes, &jException, INSTANCE, 
+                jInputStream, "org/apache/hadoop/fs/FSDataInputStream", 
+                "read", "([B)I", jbRarray) != 0) {
+        fprintf(stderr, 
+            "Call to org.apache.hadoop.fs.FSDataInputStream::read failed!\n");
+        errno = EINTERNAL;
+        noReadBytes = -1;
+    } else {
+        if(noReadBytes > 0) {
+            (*env)->GetByteArrayRegion(env, jbRarray, 0, noReadBytes, buffer);
+        }
+        //This is a valid case: there aren't any bytes left to read!
+        errno = 0;
+    }
+    (*env)->ReleaseByteArrayElements(env, jbRarray, 
+                (*env)->GetByteArrayElements(env, jbRarray, 0), JNI_ABORT);
+
+    return noReadBytes;
+}
+  
+tSize hdfsWrite(hdfsFS fs, hdfsFile f, const void* buffer, tSize length)
+{
+    // JAVA EQUIVALENT
+    // byte b[] = str.getBytes();
+    // fso.write(b);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jobject jOutputStream = (jobject)(f ? f->file : 0);
+
+    jthrowable jException;
+    jbyteArray jbWarray;
+    jint noWrittenBytes = 0;
+
+    //Sanity check
+    if (!f || f->type == UNINITIALIZED) {
+        errno = EBADF;
+        return -1;
+    }
+
+    //Error checking... make sure that this file is 'writable'
+    if (f->type != OUTPUT) {
+        fprintf(stderr, "Cannot write into a non-OutputStream object!\n");
+        errno = EINVAL;
+        return -1;
+    }
+
+    //Write the requisite bytes into the file
+    jbWarray = (*env)->NewByteArray(env, length);
+    (*env)->SetByteArrayRegion(env, jbWarray, 0, length, buffer);
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jOutputStream,
+                "org/apache/hadoop/fs/FSDataOutputStream", "write", 
+                "([B)V", jbWarray)) {
+        fprintf(stderr, 
+            "Call to org.apache.hadoop.fs.FSDataOutputStream::write failed!\n"
+            );
+        errno = EINTERNAL;
+        noWrittenBytes = -1;
+    } 
+    (*env)->ReleaseByteArrayElements(env, jbWarray, 
+                (*env)->GetByteArrayElements(env, jbWarray, 0), JNI_ABORT);
+
+    //Return no. of bytes succesfully written (libc way)
+    //i.e. 'length' itself! ;-)
+    return length;
+}
+
+int hdfsSeek(hdfsFS fs, hdfsFile f, tOffset desiredPos) 
+{
+    // JAVA EQUIVALENT
+    //  fis.seek(pos);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jobject jInputStream = (jobject)(f ? f->file : 0);
+
+    jthrowable jException;
+
+    //Sanity check
+    if (!f || f->type != INPUT) {
+        errno = EBADF;
+        return -1;
+    }
+
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jInputStream, 
+                "org/apache/hadoop/fs/FSDataInputStream", "seek", 
+                "(J)V", desiredPos) != 0) {
+        fprintf(stderr, 
+            "Call to org.apache.hadoop.fs.FSDataInputStream::seek failed!\n");
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    return 0;
+}
+
+tOffset hdfsTell(hdfsFS fs, hdfsFile f)
+{
+    // JAVA EQUIVALENT
+    //  pos = f.getPos();
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jobject jStream = (jobject)(f ? f->file : 0);
+
+    jthrowable jException;
+
+    //Sanity check
+    if (!f || f->type == UNINITIALIZED) {
+        errno = EBADF;
+        return -1;
+    }
+
+    const char* interface = (f->type == INPUT) ? 
+        "org/apache/hadoop/fs/FSDataInputStream" : 
+        "org/apache/hadoop/fs/FSDataOutputStream";
+
+    jlong currentPos  = -1;
+    if (invokeMethod(env,(RetVal*)&currentPos, &jException, INSTANCE, 
+                jStream, interface, "getPos", "()J") != 0) {
+        fprintf(stderr, "Call to org.apache.hadoop.fs.FSDataInputStream::getPos failed!\n");
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    return (tOffset)currentPos;
+}
+
+int hdfsFlush(hdfsFS fs, hdfsFile f) 
+{
+    // JAVA EQUIVALENT
+    //  fos.flush();
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jobject jOutputStream = (jobject)(f ? f->file : 0);
+
+    jthrowable jException;
+
+    //Sanity check
+    if (!f || f->type != OUTPUT) {
+        errno = EBADF;
+        return -1;
+    }
+
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jOutputStream, 
+                "org/apache/hadoop/fs/FSDataOutputStream", "flush", 
+                "()V") != 0) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FSDataInputStream::flush failed!\n"
+                );
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    return 0;
+}
+
+
+int hdfsAvailable(hdfsFS fs, hdfsFile f)
+{
+    // JAVA EQUIVALENT
+    //  fis.available();
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jobject jInputStream = (jobject)(f ? f->file : 0);
+
+    jthrowable jException;
+
+    //Sanity check
+    if (!f || f->type != INPUT) {
+        errno = EBADF;
+        return -1;
+    }
+
+    jint available = -1;
+    if (invokeMethod(env, (RetVal*)&available, &jException, INSTANCE, jInputStream, 
+                "org/apache/hadoop/fs/FSDataInputStream", "available", 
+                "()I") != 0) {
+        fprintf(stderr, 
+            "Call to org.apache.hadoop.fs.FSDataInputStream::available failed!\n"
+            );
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    return available;
+}
+
+int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
+{
+    //JAVA EQUIVALENT
+    //  FileUtil::copy(srcFS, srcPath, dstFS, dstPath, deleteSource = false, conf)
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jSrcFS = (jobject)srcFS;
+    jobject jDstFS = (jobject)dstFS;
+    jobject jSrcPath = constructNewObjectOfPath(env, src);
+    jobject jDstPath = constructNewObjectOfPath(env, dst);
+    if (jSrcPath == NULL || jDstPath == NULL) {
+        return -1;
+    }
+    jthrowable jException;
+    int retval = 0;
+
+    //Create the org.apache.hadoop.conf.Configuration object
+    jobject jConfiguration = constructNewObjectOfClass(env, &jException, 
+            "org/apache/hadoop/conf/Configuration", "()V");
+    if (jConfiguration == NULL) {
+        fprintf(stderr, 
+                "Can't construct instance of class org.apache.hadoop.conf.Configuration\n"
+                );
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    //FileUtil::copy
+    jboolean deleteSource = 0; //Only copy
+    jboolean jRetVal = 0;
+    if (invokeMethod(env, (RetVal*)&jRetVal, &jException, 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) {
+        fprintf(stderr, 
+          "Call to org.apache.hadoop.fs.FileUtil::copy failed!\n");
+        errno = EINTERNAL;
+        retval = -1;
+        goto done;
+    }
+
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jConfiguration);
+    destroyLocalReference(env, jSrcPath);
+    destroyLocalReference(env, jDstPath);
+  
+    return retval;
+}
+
+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();
+
+    //Parameters
+    jobject jSrcFS = (jobject)srcFS;
+    jobject jDstFS = (jobject)dstFS;
+    jobject jSrcPath = constructNewObjectOfPath(env, src);
+    jobject jDstPath = constructNewObjectOfPath(env, dst);
+    if (jSrcPath == NULL || jDstPath == NULL) {
+        return -1;
+    }
+    jthrowable jException;
+    int retval = 0;
+
+    //Create the org.apache.hadoop.conf.Configuration object
+    jobject jConfiguration = constructNewObjectOfClass(env, &jException, 
+            "org/apache/hadoop/conf/Configuration", "()V");
+    if (jConfiguration == NULL) {
+        fprintf(stderr, 
+                "Can't construct instance of class org.apache.hadoop.conf.Configuration\n"
+                );
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    //FileUtil::copy
+    jboolean deleteSource = 1; //Delete src after copy
+    jboolean jRetVal = 0;
+    if (invokeMethod(env, (RetVal*)&jRetVal, &jException, 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) {
+        fprintf(stderr, 
+          "Call to org.apache.hadoop.fs.FileUtil::copy(move) failed!\n");
+        errno = EINTERNAL;
+        retval = -1;
+        goto done;
+    }
+
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jConfiguration);
+    destroyLocalReference(env, jSrcPath);
+    destroyLocalReference(env, jDstPath);
+  
+    return retval;
+}
+
+int hdfsDelete(hdfsFS fs, const char* path)
+{
+    // JAVA EQUIVALENT:
+    //  File f = new File(path);
+    //  bool retval = fs.delete(f);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //Create an object of java.io.File
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return -1;
+    }
+
+    //Delete the file
+    jboolean retval = 1;
+    if (invokeMethod(env, (RetVal*)&retval, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "delete", 
+                "(Lorg/apache/hadoop/fs/Path;)Z", jPath)) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::delete failed!\n");
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return (retval) ? 0 : -1;
+}
+
+int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath)
+{
+    // JAVA EQUIVALENT:
+    //  Path old = new Path(oldPath);
+    //  Path new = new Path(newPath);
+    //  fs.rename(old, new);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //Create objects of org.apache.hadoop.fs.Path
+    jobject jOldPath = constructNewObjectOfPath(env, oldPath);
+    jobject jNewPath = constructNewObjectOfPath(env, newPath);
+    if (jOldPath == NULL || jNewPath == NULL) {
+        return -1;
+    }
+
+    //Rename the file
+    jboolean retval = 1;
+    if (invokeMethod(env, (RetVal*)&retval, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "rename", 
+                "(Lorg/apache/hadoop/fs/Path;Lorg/apache/hadoop/fs/Path;)Z", 
+                jOldPath, jNewPath)) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::rename failed!\n");
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jOldPath);
+    destroyLocalReference(env, jNewPath);
+
+    return (retval) ? 0 : -1;
+}
+
+int hdfsLock(hdfsFS fs, const char* path, int shared)
+{
+    // JAVA EQUIVALENT:
+    //  Path p = new Path(path);
+    //  fs.lock(p);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Parameters
+    jobject jFS = (jobject)fs;
+    jboolean jb_shared = shared;
+
+    jthrowable jException;
+
+    //Create an object of org.apache.hadoop.fs.Path
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return -1;
+    }
+
+    //Lock the file
+    int retval = 0;
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "lock", 
+                "(Lorg/apache/hadoop/fs/Path;Z)V", jPath, jb_shared)) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::lock failed!\n");
+        errno = EINTERNAL;
+        retval = -1;
+    }
+
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return retval;
+}
+
+int hdfsReleaseLock(hdfsFS fs, const char* path)
+{
+    // JAVA EQUIVALENT:
+    //  Path f = new Path(path);
+    //  fs.release(f);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //Create an object of java.io.File
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return -1;
+    }
+
+    //Release the lock on the file
+    int retval = 0;
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "release", 
+                "(Lorg/apache/hadoop/fs/Path;)V", jPath)) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::release failed!\n");
+        errno = EINTERNAL;
+        retval = -1;
+        goto done;
+    }
+
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return retval;
+}
+
+char* hdfsGetWorkingDirectory(hdfsFS fs, char* buffer, size_t bufferSize)
+{
+    // JAVA EQUIVALENT:
+    //  Path p = fs.getWorkingDirectory(); 
+    //  return p.toString()
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jobject jPath = NULL;
+    jthrowable jException;
+
+    //FileSystem::getWorkingDirectory()
+    if (invokeMethod(env, (RetVal*)&jPath, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "getWorkingDirectory", 
+                "()Lorg/apache/hadoop/fs/Path;") || jPath == NULL) {
+        fprintf(stderr, "Call to FileSystem::getWorkingDirectory failed!\n");
+        errno = EINTERNAL;
+        return NULL;
+    }
+
+    //Path::toString()
+    jstring jPathString;
+    if (invokeMethod(env, (RetVal*)&jPathString, &jException, INSTANCE, jPath, 
+                "org/apache/hadoop/fs/Path", "toString", "()Ljava/lang/String;")) { 
+        fprintf(stderr, "Call to Path::toString failed!\n");
+        errno = EINTERNAL;
+        destroyLocalReference(env, jPath);
+        return NULL;
+    }
+
+    //Copy to user-provided buffer
+    strncpy(buffer, (char*)(*env)->GetStringUTFChars(env, jPathString, NULL), 
+            bufferSize);
+
+    //Delete unnecessary local references
+    (*env)->ReleaseStringUTFChars(env, jPathString, 
+                                (*env)->GetStringUTFChars(env, jPathString, NULL));
+    destroyLocalReference(env, jPath);
+
+    return buffer;
+}
+
+int hdfsSetWorkingDirectory(hdfsFS fs, const char* path)
+{
+    // JAVA EQUIVALENT:
+    //  fs.setWorkingDirectory(Path(path)); 
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    int retval = 0;
+
+    //Create an object of org.apache.hadoop.fs.Path
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return -1;
+    }
+
+    //FileSystem::setWorkingDirectory()
+    if (invokeMethod(env, NULL, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "setWorkingDirectory", 
+                "(Lorg/apache/hadoop/fs/Path;)V", jPath) || jPath == NULL) {
+        fprintf(stderr, "Call to FileSystem::setWorkingDirectory failed!\n");
+        errno = EINTERNAL;
+        retval = -1;
+    }
+
+    done:
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return retval;
+}
+
+int hdfsCreateDirectory(hdfsFS fs, const char* path)
+{
+    // JAVA EQUIVALENT:
+    //  fs.mkdirs(new Path(path));
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //Create an object of org.apache.hadoop.fs.Path
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return -1;
+    }
+
+    //Create the directory
+    jboolean jRetVal = 0;
+    if (invokeMethod(env, (RetVal*)&jRetVal, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "mkdirs", 
+                "(Lorg/apache/hadoop/fs/Path;)Z", jPath)) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::mkdirs failed!\n");
+        errno = EINTERNAL;
+        goto done;
+    }
+
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return (jRetVal) ? 0 : -1;
+}
+
+char*** hdfsGetHosts(hdfsFS fs, const char* path, tOffset start, tOffset length)
+{
+    // JAVA EQUIVALENT:
+    //  fs.getFileCacheHints(new Path(path), start, length);
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //Create an object of org.apache.hadoop.fs.Path
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return NULL;
+    }
+
+    //org.apache.hadoop.fs.FileSystem::getFileCacheHints
+    char*** blockHosts = NULL;
+    jobjectArray jFileCacheHints;
+    if (invokeMethod(env, (RetVal*)&jFileCacheHints, &jException, INSTANCE, 
+                jFS, "org/apache/hadoop/fs/FileSystem", "getFileCacheHints", 
+                "(Lorg/apache/hadoop/fs/Path;JJ)[[Ljava/lang/String;", jPath, 
+                start, length)) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::getFileCacheHints failed!\n"
+               );
+        errno = EINTERNAL;
+        goto done;
+    }
+
+    //Figure out no of entries in jFileCacheHints 
+    //Allocate memory and add NULL at the end
+    jsize jNumFileBlocks = (*env)->GetArrayLength(env, jFileCacheHints);
+    blockHosts = malloc(sizeof(char**) * (jNumFileBlocks+1));
+    if (blockHosts == NULL) {
+        errno = ENOMEM;
+        goto done;
+    }
+    blockHosts[jNumFileBlocks] = NULL;
+    if (jNumFileBlocks == 0) {
+        errno = 0;
+        goto done;
+    }
+
+    //Now parse each block to get hostnames
+    int i = 0;
+    for(i=0; i < jNumFileBlocks; ++i) {
+        jobjectArray jFileBlockHosts = (*env)->GetObjectArrayElement(env, 
+                                                        jFileCacheHints, i);
+
+        //Figure out no of entries in jFileCacheHints 
+        //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;
+            goto done;
+        }
+        blockHosts[i][jNumBlockHosts] = NULL;
+
+        //Now parse each hostname
+        int j = 0;
+        for(j=0; j < jNumBlockHosts; ++j) {
+            jstring jHost = (*env)->GetObjectArrayElement(env, 
+                    jFileBlockHosts, j);
+            blockHosts[i][j] = strdup((char*)(*env)->GetStringUTFChars(env, 
+                                                jHost, NULL));
+            (*env)->ReleaseStringUTFChars(env, jHost, 
+                                (*env)->GetStringUTFChars(env, jHost, NULL));
+        }
+    }
+  
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return blockHosts;
+}
+
+tOffset hdfsGetDefaultBlockSize(hdfsFS fs)
+{
+    // JAVA EQUIVALENT:
+    //  fs.getDefaultBlockSize();
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //FileSystem::getDefaultBlockSize()
+    tOffset blockSize = -1;
+    if (invokeMethod(env, (RetVal*)&blockSize, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "getDefaultBlockSize", 
+                "()J") != 0) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::getDefaultBlockSize failed!\n"
+                );
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    return blockSize;
+}
+
+tOffset hdfsGetCapacity(hdfsFS fs)
+{
+    // JAVA EQUIVALENT:
+    //  fs.getRawCapacity();
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    if (!((*env)->IsInstanceOf(env, jFS, 
+                    globalClassReference("org/apache/hadoop/dfs/DistributedFileSystem", 
+                        env)))) {
+        fprintf(stderr, 
+                "hdfsGetCapacity works only on a DistributedFileSystem!\n");
+        return -1;
+    }
+
+    //FileSystem::getRawCapacity()
+    tOffset rawCapacity = -1;
+    if (invokeMethod(env, (RetVal*)&rawCapacity, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/dfs/DistributedFileSystem", 
+                "getRawCapacity", "()J") != 0) {
+        fprintf(stderr, 
+            "Call to org.apache.hadoop.fs.FileSystem::getRawCapacity failed!\n"
+            );
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    return rawCapacity;
+}
+  
+tOffset hdfsGetUsed(hdfsFS fs)
+{
+    // JAVA EQUIVALENT:
+    //  fs.getRawUsed();
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    if (!((*env)->IsInstanceOf(env, jFS, 
+                    globalClassReference("org/apache/hadoop/dfs/DistributedFileSystem", 
+                        env)))) {
+        fprintf(stderr, 
+                "hdfsGetUsed works only on a DistributedFileSystem!\n");
+        return -1;
+    }
+
+    //FileSystem::getRawUsed()
+    tOffset rawUsed = -1;
+    if (invokeMethod(env, (RetVal*)&rawUsed, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/dfs/DistributedFileSystem", "getRawUsed", 
+                "()J") != 0) {
+        fprintf(stderr, 
+            "Call to org.apache.hadoop.fs.FileSystem::getRawUsed failed!\n");
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    return rawUsed;
+}
+ 
+static int getFileInfo(JNIEnv *env, jobject jFS, jobject jPath, hdfsFileInfo *fileInfo)
+{
+    // JAVA EQUIVALENT:
+    //  fs.isDirectory(f)
+    //  fs.lastModified() ??
+    //  fs.getLength(f)
+    //  f.getPath()
+
+    jthrowable jException;
+
+    jboolean jIsDir;
+    if (invokeMethod(env, (RetVal*)&jIsDir, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "isDirectory", 
+                "(Lorg/apache/hadoop/fs/Path;)Z", jPath) != 0) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::isDirectory failed!\n"
+                );
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    /*
+    jlong jModTime = 0;
+    if (invokeMethod(env, (RetVal*)&jModTime, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "lastModified", 
+                "(Lorg/apache/hadoop/fs/Path;)J", jPath) != 0) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::lastModified failed!\n"
+                );
+        errno = EINTERNAL;
+        return -1;
+    }
+    */
+
+    jlong jFileLength = 0;
+    if (!jIsDir) {
+        if (invokeMethod(env, (RetVal*)&jFileLength, &jException, INSTANCE, 
+                    jFS, "org/apache/hadoop/fs/FileSystem", "getLength", 
+                    "(Lorg/apache/hadoop/fs/Path;)J", jPath) != 0) {
+            fprintf(stderr, 
+                    "Call to org.apache.hadoop.fs.FileSystem::getLength failed!\n"
+                    );
+            errno = EINTERNAL;
+            return -1;
+        }
+    }
+
+    jstring jPathName;
+    if (invokeMethod(env, (RetVal*)&jPathName, &jException, INSTANCE, jPath, 
+                "org/apache/hadoop/fs/Path", "toString", "()Ljava/lang/String;")) { 
+        fprintf(stderr, "Call to org.apache.hadoop.fs.Path::toString failed!\n");
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    fileInfo->mKind = (jIsDir ? kObjectKindDirectory : kObjectKindFile);
+    //fileInfo->mCreationTime = jModTime;
+    fileInfo->mSize = jFileLength;
+    fileInfo->mName = strdup((char*)(*env)->GetStringUTFChars(env, 
+                jPathName, NULL));
+
+    (*env)->ReleaseStringUTFChars(env, jPathName,
+                               (*env)->GetStringUTFChars(env, jPathName, NULL));
+
+    return 0;
+}
+
+hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries)
+{
+    // JAVA EQUIVALENT:
+    //  Path p(path);
+    //  Path []pathList = fs.listPaths(p)
+    //  foreach path in pathList 
+    //    getFileInfo(path)
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //Create an object of org.apache.hadoop.fs.Path
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return NULL;
+    }
+
+    hdfsFileInfo *pathList = 0; 
+
+    jobjectArray jPathList;
+    if (invokeMethod(env, (RetVal*)&jPathList, &jException, INSTANCE, jFS, 
+                "org/apache/hadoop/fs/FileSystem", "listPaths", 
+                "(Lorg/apache/hadoop/fs/Path;)[Lorg/apache/hadoop/fs/Path;", jPath) != 0) {
+        fprintf(stderr, 
+                "Call to org.apache.hadoop.fs.FileSystem::listPaths failed!\n"
+                );
+        errno = EINTERNAL;
+        goto done;
+    }
+
+    //Figure out no of entries in that directory
+    jsize jPathListSize = (*env)->GetArrayLength(env, jPathList);
+    *numEntries = jPathListSize;
+    if (jPathListSize == 0) {
+        errno = 0;
+        goto done;
+    }
+
+    //Allocate memory
+    pathList = malloc(sizeof(hdfsFileInfo) * jPathListSize);
+    if (pathList == NULL) {
+        errno = ENOMEM;
+        goto done;
+    }
+
+    //Save path information in pathList
+    jsize i;
+    for(i=0; i < jPathListSize; ++i) {
+        if (getFileInfo(env, jFS, (*env)->GetObjectArrayElement(env, 
+                        jPathList, i), &pathList[i])) {
+            errno = EINTERNAL;
+            free(pathList);
+            goto done;
+        }
+    }
+
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return pathList;
+}
+
+hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path)
+{
+    // JAVA EQUIVALENT:
+    //  File f(path);
+    //  fs.isDirectory(f)
+    //  fs.lastModified() ??
+    //  fs.getLength(f)
+    //  f.getPath()
+
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    jobject jFS = (jobject)fs;
+    jthrowable jException;
+
+    //Create an object of org.apache.hadoop.fs.Path
+    jobject jPath = constructNewObjectOfPath(env, path);
+    if (jPath == NULL) {
+        return NULL;
+    }
+
+    hdfsFileInfo *fileInfo = malloc(sizeof(hdfsFileInfo));
+    bzero(fileInfo, sizeof(hdfsFileInfo));
+    if (getFileInfo(env, jFS, jPath, fileInfo)) {
+        hdfsFreeFileInfo(fileInfo, 1);
+        fileInfo = NULL;
+        goto done;
+    }
+
+    done:
+
+    //Delete unnecessary local references
+    destroyLocalReference(env, jPath);
+
+    return fileInfo;
+}
+
+void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)
+{
+    //Free the mName
+    int i;
+    for (i=0; i < numEntries; ++i) {
+        if (hdfsFileInfo[i].mName) {
+            free(hdfsFileInfo[i].mName);
+        }
+    }
+
+    //Free entire block
+    free(hdfsFileInfo);
+}
+
+jobject hdfsConvertToGlobalRef(jobject localRef)
+{
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Create the global reference
+    jobject globalRef = (*env)->NewGlobalRef(env, localRef);
+    if(globalRef == NULL) {
+        (*env)->ExceptionDescribe(env);
+        return NULL; 
+    }
+
+    //Destroy the local reference
+    (*env)->DeleteLocalRef(env, globalRef);
+
+    return globalRef;
+}
+
+void hdfsDeleteGlobalRef(jobject globalRef)
+{
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+
+    //Destroy the global reference
+    (*env)->DeleteGlobalRef(env, globalRef);
+}
+
+/**
+ * vim: ts=4: sw=4: et:
+ */

Added: lucene/hadoop/trunk/src/c++/libhdfs/hdfs.h
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/hdfs.h?rev=410637&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfs.h (added)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfs.h Wed May 31 12:08:38 2006
@@ -0,0 +1,322 @@
+/**
+ * Copyright 2005 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.
+ */
+
+#ifndef hdfs_H
+#define hdfs_H
+
+#include <stdio.h>
+#include <stdint.h>
+#include <string.h>
+#include <stdlib.h>
+#include <time.h>
+#include <errno.h>
+
+#include <jni.h>
+
+#define O_RDONLY 1 
+#define O_WRONLY 2
+#define EINTERNAL 255 
+
+/** All APIs set errno to meaningful values */
+#ifdef __cplusplus
+extern  "C" {
+#endif
+
+    typedef int32_t   tSize; /// size of data for read/write io ops 
+    typedef time_t    tTime; /// time type
+    typedef int64_t   tOffset;/// offset within the file
+    typedef uint16_t  tPort; /// port
+    typedef enum tObjectKind {
+        kObjectKindFile = 'F',
+        kObjectKindDirectory = 'D',
+    } tObjectKind;
+
+
+    /**
+     * The C reflection of org.apache.org.hadoop.FileSystem .
+     */
+    typedef void* hdfsFS;
+    
+    /**
+     * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
+     */
+    enum hdfsStreamType
+    {
+        UNINITIALIZED = 0,
+        INPUT = 1,
+        OUTPUT = 2,
+    };
+    struct hdfsFile_internal {
+        void* file;
+        enum hdfsStreamType type;
+    };
+    typedef struct hdfsFile_internal* hdfsFile;
+      
+    /** hdfsConnect - Connect to a hdfs file system.
+     * @param host: A string containing either a host name, or an ip address of the namenode of a hdfs cluster. 'host' should be passed as NULL if you want to connect to local filesystem. 'host' should be passed as 'default' (and port as 0) to used the 'configured' filesystem (hadoop-site/hadoop-default.xml).
+     * @param port: The port on which the server is listening.
+     * @return Returns a handle to the filesystem or NULL on error.
+     */
+    hdfsFS hdfsConnect(const char* host, tPort port);
+
+    /** hdfsDisconnect - Disconnect from the hdfs file system.
+     * @param fs: The configured filesystem handle.
+     * @return Returns 0 on success, -1 on error.  
+     */
+    int hdfsDisconnect(hdfsFS fs);
+        
+    /** 
+     * hdfsOpenFile - Open a hdfs file in given mode.
+     * @param path: The full path to the file.
+     * @param flags: Either O_RDONLY or O_WRONLY, for read-only or write-only.
+     * @param bufferSize: Size of buffer for read/write - pass 0 if you want to use the default configured values.
+     * @param replication: Block replication - pass 0 if you want to use the default configured values.
+     * @param blocksize: Size of block - pass 0 if you want to use the default configured values.
+     * @return Returns the handle to the open file or NULL on error.
+     */
+    hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
+            int bufferSize, short replication, tSize blocksize);
+
+    /** 
+     * hdfsCloseFile - Close an open file. 
+     * @param fs: The configured filesystem handle.
+     * @param file: The file handle.
+     * @return Returns 0 on success, -1 on error.  
+     */
+    int hdfsCloseFile(hdfsFS fs, hdfsFile file);
+
+    /** 
+     * hdfsSeek - Seek to given offset in file. 
+     * This works only for files opened in read-only mode. 
+     * @param fs: The configured filesystem handle.
+     * @param file: The file handle.
+     * @return Returns 0 on success, -1 on error.  
+     */
+    int hdfsSeek(hdfsFS fs, hdfsFile f, tOffset desiredPos); 
+
+    /** 
+     * hdfsTell - Get the current offset in the file, in bytes.
+     * @param fs: The configured filesystem handle.
+     * @param file: The file handle.
+     * @return Current offset, -1 on error.
+     */
+    tOffset hdfsTell(hdfsFS fs, hdfsFile f);
+
+    /** 
+     * hdfsRead - Read data from an open file.
+     * @param fs: The configured filesystem handle.
+     * @param file: The file handle.
+     * @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.
+     */
+    tSize hdfsRead(hdfsFS fs, hdfsFile f, void* buffer, tSize length);
+
+    /** 
+     * hdfsWrite - Write data into an open file.
+     * @param fs: The configured filesystem handle.
+     * @param file: The file handle.
+     * @param buffer: The data.
+     * @param length: The no. of bytes to write. 
+     * @return Returns the number of bytes written, -1 on error.
+     */
+    tSize hdfsWrite(hdfsFS fs, hdfsFile f, const void* buffer, tSize length);
+
+    /** 
+     * hdfsWrite - Flush the data. 
+     * @param fs: The configured filesystem handle.
+     * @param file: The file handle.
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsFlush(hdfsFS fs, hdfsFile f);
+
+    /**
+     * hdfsAvailable - Number of bytes that can be read from this input stream without blocking.
+     * @param fs: The configured filesystem handle.
+     * @param file: The file handle.
+     * @return Returns available bytes; -1 on error. 
+     */
+    int hdfsAvailable(hdfsFS fs, hdfsFile f);
+
+    /**
+     * hdfsCopy - Copy file from one filesystem to another.
+     * @param srcFS: The handle to source filesystem.
+     * @param src: The path of source file. 
+     * @param dstFS: The handle to destination filesystem.
+     * @param dst: The path of destination file. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
+
+    /**
+     * hdfsMove - Move file from one filesystem to another.
+     * @param srcFS: The handle to source filesystem.
+     * @param src: The path of source file. 
+     * @param dstFS: The handle to destination filesystem.
+     * @param dst: The path of destination file. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
+
+    /**
+     * hdfsDelete - Delete file. 
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the file. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsDelete(hdfsFS fs, const char* path);
+
+    /**
+     * hdfsDelete - Rename file. 
+     * @param fs: The configured filesystem handle.
+     * @param oldPath: The path of the source file. 
+     * @param newPath: The path of the destination file. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath);
+
+    /**
+     * hdfsLock - Obtain a lock on the file.
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the file. 
+     * @param shared: Shared/exclusive lock-type. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsLock(hdfsFS fs, const char* path, int shared);
+
+    /**
+     * hdfsReleaseLock - Release the lock.
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the file. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsReleaseLock(hdfsFS fs, const char* path);
+
+    /** 
+     * hdfsGetWorkingDirectory - Get the current working directory for the given filesystem.
+     * @param fs: The configured filesystem handle.
+     * @param buffer: The user-buffer to copy path of cwd into. 
+     * @param bufferSize: The length of user-buffer.
+     * @return Returns buffer, NULL on error.
+     */
+    char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize);
+
+    /** 
+     * hdfsSetWorkingDirectory - Set the working directory. All relative paths will be resolved relative to it.
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the new 'cwd'. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsSetWorkingDirectory(hdfsFS fs, const char* path);
+
+    /** 
+     * hdfsCreateDirectory - Make the given file and all non-existent parents into directories.
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the directory. 
+     * @return Returns 0 on success, -1 on error. 
+     */
+    int hdfsCreateDirectory(hdfsFS fs, const char* path);
+
+    /** 
+     * hdfsFileInfo - Information about a file/directory.
+     */
+    typedef struct  {
+        tObjectKind mKind; /*file or directory */
+        char *mName; /*the name of the file */
+        tTime mCreationTime; /*the creation time for the file*/
+        tOffset mSize; /*the size of the file in bytes */
+        int replicaCount; /*the count of replicas */
+    } hdfsFileInfo;
+
+    /** 
+     * hdfsListDirectory - Get list of files/directories for a given directory-path. freehdfsFileInfo should be called to deallocate memory. 
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the directory. 
+     * @param numEntries: Set to the number of files/directories in path.
+     * @return Returns a dynamically-allocated array of hdfsFileInfo objects; NULL on error.
+     */
+    hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries);
+
+    /** 
+     * hdfsGetPathInfo - Get information about a path as a (dynamically allocated) single hdfsFileInfo struct. freehdfsFileInfo should be called when the pointer is no longer needed.
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the file. 
+     * @return Returns a dynamically-allocated hdfsFileInfo object; NULL on error.
+     */
+    hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path);
+
+    /** free up the hdfsFileInfo array (including the fields) */
+    void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries);
+
+    /** 
+     * hdfsGetHosts - Get hostnames where a particular block (determined by pos & blocksize) of a file is stored. The last element in the array is NULL. Due to replication, a single block could be present on multiple hosts.
+     * @param fs: The configured filesystem handle.
+     * @param path: The path of the file. 
+     * @param start: The start of the block.
+     * @param length: The length of the block.
+     * @return Returns a dynamically-allocated 2-d array of blocks-hosts; NULL on error.
+     */
+    char*** hdfsGetHosts(hdfsFS fs, const char* path, 
+            tOffset start, tOffset length);
+
+    /** 
+     * hdfsGetDefaultBlockSize - Get the optimum blocksize.
+     * @param fs: The configured filesystem handle.
+     * @return Returns the blocksize; -1 on error. 
+     */
+    tOffset hdfsGetDefaultBlockSize(hdfsFS fs);
+
+    /** 
+     * hdfsGetCapacity - Return the raw capacity of the filesystem.  
+     * @param fs: The configured filesystem handle.
+     * @return Returns the raw-capacity; -1 on error. 
+     */
+    tOffset hdfsGetCapacity(hdfsFS fs);
+
+    /** 
+     * hdfsGetUsed - Return the total raw size of all files in the filesystem.
+     * @param fs: The configured filesystem handle.
+     * @return Returns the total-size; -1 on error. 
+     */
+    tOffset hdfsGetUsed(hdfsFS fs);
+
+    /**
+     * hdfsConvertToGlobalRef - Return a global reference for the jobject.
+     * The user needs to explicitly call this to share the jobject between
+     * multiple threads! This function automatically deletes the local reference
+     * if it succesfully converted it to a global reference.
+     * @param localRef: The local reference which needs to be globalized.
+     * @return Returns the global reference; NULL on error.
+     */
+    jobject hdfsConvertToGlobalRef(jobject localRef);
+    
+    /**
+     * hdfsDeleteGlobalRef - Destroy a global reference.
+     * multiple threads!
+     * @param globalRef: The global reference to be destroyed.
+     * @return None. 
+     */
+    void hdfsDeleteGlobalRef(jobject globalRef);
+    
+#ifdef __cplusplus
+}
+#endif
+
+#endif /*hdfs_H*/
+
+/**
+ * vim: ts=4: sw=4:
+ */

Added: lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.h
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/hdfsJniHelper.h?rev=410637&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.h (added)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfsJniHelper.h Wed May 31 12:08:38 2006
@@ -0,0 +1,330 @@
+/**
+ * Copyright 2005 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 <jni.h>
+#include <stdio.h>
+
+#include <stdlib.h>
+#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)
+
+#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 { 
+    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
+ * methSignature: the signature of the method "(arg-types)ret-type"
+ * methType: The type of the method (STATIC or INSTANCE)
+ * instObj: Required if the methType is INSTANCE. The object to invoke
+   the method on.
+ * env: The JNIEnv pointer
+ * retval: The pointer to a union type which will contain the result of the
+   method invocation, e.g. if the method returns an Object, retval will be
+   set to that, if the method returns boolean, retval will be set to the
+   value (JNI_TRUE or JNI_FALSE), etc.
+ * exc: If the methods throws any exception, this will contain the reference
+ * Arguments (the method arguments) must be passed after methSignature
+ * 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, 
+                 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, jthrowable *exc, 
+                                  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();
+}
+
+/**
+ * vim: ts=4: sw=4: et:
+ */
+

Added: lucene/hadoop/trunk/src/c++/libhdfs/hdfs_read.c
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/c%2B%2B/libhdfs/hdfs_read.c?rev=410637&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfs_read.c (added)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfs_read.c Wed May 31 12:08:38 2006
@@ -0,0 +1,65 @@
+/**
+ * Copyright 2005 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 "hdfs.h" 
+
+int main(int argc, char **argv) {
+
+    if (argc != 4) {
+        fprintf(stderr, "Usage: hdfs_read <filename> <filesize> <buffersize>\n");
+        exit(-1);
+    }
+    
+    hdfsFS fs = hdfsConnect("default", 0);
+    if (!fs) {
+        fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
+        exit(-1);
+    } 
+ 
+    const char* rfile = argv[1];
+    tSize fileTotalSize = strtoul(argv[2], NULL, 10);
+    tSize bufferSize = strtoul(argv[3], NULL, 10);
+   
+    hdfsFile readFile = hdfsOpenFile(fs, rfile, O_RDONLY, bufferSize, 0, 0);
+    if (!readFile) {
+        fprintf(stderr, "Failed to open %s for writing!\n", rfile);
+        exit(-2);
+    }
+
+    // data to be written to the file
+    char* buffer = malloc(sizeof(char) * bufferSize);
+    if(buffer == NULL) {
+        return -2;
+    }
+    
+    // read from the file
+    tSize curSize = bufferSize;
+    for (; curSize == bufferSize;) {
+        curSize = hdfsRead(fs, readFile, (void*)buffer, curSize);
+    }
+    
+
+    free(buffer);
+    hdfsCloseFile(fs, readFile);
+    hdfsDisconnect(fs);
+
+    return 0;
+}
+
+/**
+ * vim: ts=4: sw=4: et:
+ */
+

Added: 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?rev=410637&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/libhdfs/hdfs_test.c (added)
+++ lucene/hadoop/trunk/src/c++/libhdfs/hdfs_test.c Wed May 31 12:08:38 2006
@@ -0,0 +1,192 @@
+/**
+ * Copyright 2005 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 "hdfs.h" 
+
+int main(int argc, char **argv) {
+
+    hdfsFS fs = hdfsConnect("default", 0);
+    if(!fs) {
+        fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
+        exit(-1);
+    } 
+ 
+    hdfsFS lfs = hdfsConnect(NULL, 0);
+    if(!fs) {
+        fprintf(stderr, "Oops! Failed to connect to 'local' hdfs!\n");
+        exit(-1);
+    } 
+ 
+    {
+        //Write tests
+        
+        const char* writePath = "/tmp/testfile.txt";
+        
+        hdfsFile writeFile = hdfsOpenFile(fs, writePath, O_WRONLY, 0, 0, 0);
+        if(!writeFile) {
+            fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
+
+        char* buffer = "Hello, World!";
+        tSize num_written_bytes = hdfsWrite(fs, writeFile, (void*)buffer, strlen(buffer)+1);
+        fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
+
+        tOffset currentPos = -1;
+        if ((currentPos = hdfsTell(fs, writeFile)) == -1) {
+            fprintf(stderr, 
+                    "Failed to get current file position correctly! Got %d!\n",
+                    currentPos);
+            exit(-1);
+        }
+        fprintf(stderr, "Current position: %ld\n", currentPos);
+
+        if (hdfsFlush(fs, writeFile)) {
+            fprintf(stderr, "Failed to 'flush' %s\n", writePath); 
+            exit(-1);
+        }
+        fprintf(stderr, "Flushed %s successfully!\n", writePath); 
+
+        hdfsCloseFile(fs, writeFile);
+    }
+
+    {
+        //Read tests
+        
+        const char* readPath = "/tmp/testfile.txt";
+        hdfsFile readFile = hdfsOpenFile(fs, readPath, O_RDONLY, 0, 0, 0);
+        if (!readFile) {
+            fprintf(stderr, "Failed to open %s for reading!\n", readPath);
+            exit(-1);
+        }
+
+        fprintf(stderr, "hdfsAvailable: %d\n", hdfsAvailable(fs, readFile));
+
+        tOffset seekPos = 1;
+        if(hdfsSeek(fs, readFile, seekPos)) {
+            fprintf(stderr, "Failed to seek %s for reading!\n", readPath);
+            exit(-1);
+        }
+
+        tOffset currentPos = -1;
+        if((currentPos = hdfsTell(fs, readFile)) != seekPos) {
+            fprintf(stderr, 
+                    "Failed to get current file position correctly! Got %d!\n", 
+                    currentPos);
+            exit(-1);
+        }
+        fprintf(stderr, "Current position: %ld\n", currentPos);
+
+        static char buffer[32];
+        tSize num_read_bytes = hdfsRead(fs, readFile, (void*)buffer, 
+                sizeof(buffer));
+        fprintf(stderr, "Read following %d bytes:\n%s\n", 
+                num_read_bytes, buffer);
+
+        hdfsCloseFile(fs, readFile);
+    }
+
+ 
+    {
+        //Generic file-system operations
+
+        const char* srcPath = "/tmp/testfile.txt";
+        const char* dstPath = "/tmp/testfile2.txt";
+
+        fprintf(stderr, "hdfsCopy(remote-local): %s\n", (hdfsCopy(fs, srcPath, lfs, srcPath) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsCopy(remote-remote): %s\n", (hdfsCopy(fs, srcPath, fs, dstPath) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsMove(local-local): %s\n", (hdfsMove(lfs, srcPath, lfs, dstPath) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsMove(remote-local): %s\n", (hdfsMove(fs, srcPath, lfs, srcPath) ? "Failed!" : "Success!"));
+
+        fprintf(stderr, "hdfsRename: %s\n", (hdfsRename(fs, dstPath, srcPath) ? "Failed!" : "Success!"));
+
+        fprintf(stderr, "hdfsLock: %s\n", (hdfsLock(fs, srcPath, 1) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsReleaseLock: %s\n", (hdfsReleaseLock(fs, srcPath) ? "Failed!" : "Success!"));
+
+        const char* slashTmp = "/tmp";
+        const char* newDirectory = "/tmp/newdir";
+        fprintf(stderr, "hdfsCreateDirectory: %s\n", (hdfsCreateDirectory(fs, newDirectory) ? "Failed!" : "Success!"));
+
+        char buffer[256];
+        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", (hdfsGetWorkingDirectory(fs, buffer, sizeof(buffer)) ? buffer : "Failed!"));
+        fprintf(stderr, "hdfsSetWorkingDirectory: %s\n", (hdfsSetWorkingDirectory(fs, slashTmp) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", (hdfsGetWorkingDirectory(fs, buffer, sizeof(buffer)) ? buffer : "Failed!"));
+
+        fprintf(stderr, "hdfsGetDefaultBlockSize: %Ld\n", hdfsGetDefaultBlockSize(fs));
+        fprintf(stderr, "hdfsGetCapacity: %Ld\n", hdfsGetCapacity(fs));
+        fprintf(stderr, "hdfsGetUsed: %Ld\n", hdfsGetUsed(fs));
+
+        hdfsFileInfo *fileInfo = NULL;
+        if(fileInfo = hdfsGetPathInfo(fs, slashTmp)) {
+            fprintf(stderr, "Yaay! hdfsGetPathInfo - SUCCESS!\n");
+            fprintf(stderr, "Name: %s,", fileInfo->mName);
+            fprintf(stderr, "Type: %c,", (char)fileInfo->mKind);
+            fprintf(stderr, "Size: %ld\n", fileInfo->mSize);
+            hdfsFreeFileInfo(fileInfo, 1);
+        } else {
+            fprintf(stderr, "waah! hdfsGetPathInfo for %s - FAILED!\n", slashTmp);
+        }
+
+        hdfsFileInfo *fileList = 0;
+        int numEntries = 0;
+        if(fileList = hdfsListDirectory(fs, slashTmp, &numEntries)) {
+            int i = 0;
+            for(i=0; i < numEntries; ++i) {
+                fprintf(stderr, "Name: %s,", fileList[i].mName);
+                fprintf(stderr, "Type: %c,", (char)fileList[i].mKind);
+                fprintf(stderr, "Size: %ld\n", fileList[i].mSize);
+            }
+            hdfsFreeFileInfo(fileList, numEntries);
+        } else {
+            if (errno) {
+                fprintf(stderr, "waah! hdfsListDirectory - FAILED!\n");
+            } else {
+                fprintf(stderr, "Empty directory!\n");
+            }
+        }
+
+        char*** hosts = hdfsGetHosts(fs, srcPath, 0, 1);
+        if(hosts) {
+            fprintf(stderr, "hdfsGetHosts - SUCCESS! ... \n");
+            int i=0; 
+            while(hosts[i]) {
+                int j = 0;
+                while(hosts[i][j]) {
+                    fprintf(stderr, 
+                            "\thosts[%d][%d] - %s\n", i, j, hosts[i][j]);
+                    ++j;
+                }
+                ++i;
+            }
+        } else {
+            fprintf(stderr, "waah! hdfsGetHosts - FAILED!\n");
+        }
+        
+        // Clean up
+        fprintf(stderr, "hdfsDelete: %s\n", (hdfsDelete(fs, newDirectory) ? "Failed!" : "Success!"));
+        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!"));
+    }
+
+    return 0;
+}
+
+/**
+ * vim: ts=4: sw=4: et:
+ */
+