You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by el...@apache.org on 2011/06/20 18:03:12 UTC

svn commit: r1137675 - in /hadoop/common/trunk/hdfs: ./ src/c++/libhdfs/ src/contrib/ src/contrib/fuse-dfs/ src/contrib/fuse-dfs/src/

Author: eli
Date: Mon Jun 20 16:03:11 2011
New Revision: 1137675

URL: http://svn.apache.org/viewvc?rev=1137675&view=rev
Log:
HDFS-420. Fuse-dfs should cache fs handles. Contributed by Brian Bockelman and Eli Collins

Modified:
    hadoop/common/trunk/hdfs/CHANGES.txt
    hadoop/common/trunk/hdfs/src/c++/libhdfs/hdfs.c
    hadoop/common/trunk/hdfs/src/contrib/build-contrib.xml
    hadoop/common/trunk/hdfs/src/contrib/build.xml
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/build.xml
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/configure.ac
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.h
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_context_handle.h
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.h
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_access.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chmod.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_getattr.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_mkdir.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_open.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_readdir.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rename.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_statfs.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_truncate.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_unlink.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_utimens.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_init.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_options.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_stat_struct.c
    hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_users.c

Modified: hadoop/common/trunk/hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/CHANGES.txt?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hdfs/CHANGES.txt Mon Jun 20 16:03:11 2011
@@ -514,6 +514,8 @@ Trunk (unreleased changes)
 
     HDFS-2073. Add @Override annotation to NameNode. (suresh)
 
+    HDFS-420. Fuse-dfs should cache fs handles. (Brian Bockelman and eli)
+
   OPTIMIZATIONS
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

Modified: hadoop/common/trunk/hdfs/src/c++/libhdfs/hdfs.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/c%2B%2B/libhdfs/hdfs.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/c++/libhdfs/hdfs.c (original)
+++ hadoop/common/trunk/hdfs/src/c++/libhdfs/hdfs.c Mon Jun 20 16:03:11 2011
@@ -259,7 +259,7 @@ hdfsFS hdfsConnectAsUser(const char* hos
       sprintf(cURI, "hdfs://%s:%d", host, (int)(port));
       if (cURI == NULL) {
         fprintf (stderr, "Couldn't allocate an object of size %d",
-                 strlen(host) + 16);
+                 (int)(strlen(host) + 16));
         errno = EINTERNAL;			
         goto done;	
       }

Modified: hadoop/common/trunk/hdfs/src/contrib/build-contrib.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/build-contrib.xml?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/build-contrib.xml (original)
+++ hadoop/common/trunk/hdfs/src/contrib/build-contrib.xml Mon Jun 20 16:03:11 2011
@@ -43,6 +43,8 @@
   <property name="test.output" value="no"/>
   <property name="test.timeout" value="900000"/>
   <property name="build.dir" location="${hadoop.root}/build/contrib/${name}"/>
+  <property name="build.webapps.root.dir" value="${hadoop.root}/build/web"/>
+  <property name="build.webapps" value="${build.webapps.root.dir}/webapps"/>
   <property name="build.classes" location="${build.dir}/classes"/>
   <!-- NB: sun.arch.data.model is not supported on all platforms -->
   <property name="build.platform"
@@ -96,7 +98,8 @@
 
   <!-- the unit test classpath -->
   <path id="test.classpath">
-    <pathelement location="${build.test}" />
+    <pathelement location="${build.test}"/>
+    <pathelement location="${build.webapps.root.dir}"/>
     <pathelement location="${hadoop.root}/build/test/core/classes"/>
     <pathelement location="${hadoop.root}/build/test/hdfs/classes"/>
     <pathelement location="${hadoop.root}/build/test/mapred/classes"/>

Modified: hadoop/common/trunk/hdfs/src/contrib/build.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/build.xml?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/build.xml (original)
+++ hadoop/common/trunk/hdfs/src/contrib/build.xml Mon Jun 20 16:03:11 2011
@@ -28,7 +28,7 @@
   <!-- ====================================================== -->
   <target name="compile">
     <subant target="compile">
-      <fileset dir="." includes="hdfsproxy/build.xml"/>
+      <fileset dir="." includes="*/build.xml"/>
     </subant>
   </target>
   
@@ -45,6 +45,10 @@
   <!-- Test all the contribs.                               -->
   <!-- ====================================================== -->
   <target name="test">
+    <subant target="test">
+      <fileset dir="." includes="fuse-dfs/build.xml"/>
+    </subant> 
+
     <!-- hdfsproxy tests failing due to HDFS-1666
     <subant target="test">
       <fileset dir="." includes="hdfsproxy/build.xml"/>

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/build.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/build.xml?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/build.xml (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/build.xml Mon Jun 20 16:03:11 2011
@@ -29,11 +29,7 @@
     </fail>
   </target>
 
-  <target name="compile">
-    <condition property="perms" value="1" else="0">
-      <not> <isset property="libhdfs.noperms"/> </not>
-    </condition>
-
+  <target name="compile" if="fusedfs">
     <exec executable="autoreconf" dir="${basedir}" 
           searchpath="yes" failonerror="yes">
        <arg value="-if"/>
@@ -49,7 +45,18 @@
       <env key="HADOOP_PREFIX" value="${hadoop.root}"/>
       <env key="PACKAGE_VERSION" value="0.1.0"/>
       <env key="BUILD_PLATFORM" value="${build.platform}" />
-      <env key="PERMS" value="${perms}"/>
+    </exec>
+
+    <mkdir dir="${build.dir}"/>
+    <mkdir dir="${build.dir}/test"/>
+
+    <!-- Use exec since the copy task doesn't preserve attrs -->
+    <exec executable="cp" failonerror="true">
+      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs ${build.dir}"/>
+    </exec>
+
+    <exec executable="cp" failonerror="true">
+      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh ${build.dir}"/>
     </exec>
   </target>
 
@@ -57,8 +64,6 @@
   <target name="package" />
 
   <target name="compile-test" depends="ivy-retrieve-common, check-libhdfs-exists" if="fusedfs">
-    <mkdir dir="${build.dir}"/>
-    <mkdir dir="${build.dir}/test"/>
     <javac encoding="${build.encoding}"
 	   srcdir="${src.test}"
 	   includes="**/*.java"
@@ -66,28 +71,18 @@
 	   debug="${javac.debug}">
       <classpath refid="test.classpath"/>
     </javac>
-
-    <!-- Use exec since the copy task doesn't preserve attrs -->
-    <exec executable="cp" failonerror="true">
-      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs ${build.dir}"/>
-    </exec>
-
-    <mkdir dir="${build.dir}/test"/>
-
-    <exec executable="cp" failonerror="true">
-      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh ${build.dir}"/>
-    </exec>
   </target>
 
   <target name="test" depends="compile-test,check-libhdfs-exists" if="fusedfs">
-    <junit showoutput="${test.output}" fork="yes" printsummary="yes" errorProperty="tests.failed" haltonfailure="no" failureProperty="tests.failed">
+    <junit showoutput="${test.output}" fork="yes" printsummary="yes"
+           errorProperty="tests.failed" haltonfailure="no" failureProperty="tests.failed">
       <classpath refid="test.classpath"/>
       <sysproperty key="test.build.data" value="${build.test}/data"/>
       <sysproperty key="build.test" value="${build.test}"/>
       <sysproperty key="user.dir" value="${build.test}/data"/>
       <sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/>
       <sysproperty key="test.src.dir" value="${test.src.dir}"/>
-      <formatter type="plain" />
+      <formatter type="${test.junit.output.format}" />
       <batchtest todir="${build.test}" unless="testcase">
         <fileset dir="${src.test}">
           <include name="**/Test*.java"/>

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/configure.ac
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/configure.ac?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/configure.ac (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/configure.ac Mon Jun 20 16:03:11 2011
@@ -42,6 +42,9 @@ AC_PREFIX_DEFAULT([`pwd`])
  DEFS=""
 AC_SUBST([DEFS])
 
+# Need GNU source for multiple hashtables from glibc
+AC_GNU_SOURCE
+
 AC_FUNC_GETGROUPS
 AC_TYPE_GETGROUPS
 

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.c Mon Jun 20 16:03:11 2011
@@ -17,37 +17,106 @@
  */
 
 #include "hdfs.h"
-
 #include "fuse_dfs.h"
 #include "fuse_connect.h"
 #include "fuse_users.h" 
 
+#include <search.h>
+
+#define MAX_ELEMENTS (16 * 1024)
+static struct hsearch_data *fsTable = NULL;
+static pthread_mutex_t tableMutex = PTHREAD_MUTEX_INITIALIZER;
+
+/*
+ * Allocate a hash table for fs handles. Returns 0 on success,
+ * -1 on failure.
+ */
+int allocFsTable(void) {
+  assert(NULL == fsTable);
+  fsTable = calloc(1, sizeof(struct hsearch_data));
+  if (0 == hcreate_r(MAX_ELEMENTS, fsTable)) {
+    ERROR("Unable to initialize connection table");
+    return -1;
+  }
+  return 0;
+}
+
+/*
+ * Find a fs handle for the given key. Returns a fs handle, 
+ * or NULL if there is no fs for the given key.
+ */
+static hdfsFS findFs(char *key) {
+  ENTRY entry;
+  ENTRY *entryP = NULL;
+  entry.key = key;
+  if (0 == hsearch_r(entry, FIND, &entryP, fsTable)) {
+    return NULL;
+  }
+  assert(NULL != entryP->data);
+  return (hdfsFS)entryP->data;
+}
 
-#if PERMS
+/*
+ * Insert the given fs handle into the table.
+ * Returns 0 on success, -1 on failure.
+ */
+static int insertFs(char *key, hdfsFS fs) {
+  ENTRY entry;
+  ENTRY *entryP = NULL;
+  assert(NULL != fs);
+  entry.key = strdup(key);
+  if (entry.key == NULL) {
+    return -1;
+  }
+  entry.data = (void*)fs;
+  if (0 == hsearch_r(entry, ENTER, &entryP, fsTable)) {
+    return -1;
+  }
+  return 0;
+}
 
-/**
- * Connects to the NN as the current user/group according to FUSE
- *
+/*
+ * Connect to the NN as the current user/group.
+ * Returns a fs handle on success, or NULL on failure.
  */
 hdfsFS doConnectAsUser(const char *hostname, int port) {
   uid_t uid = fuse_get_context()->uid;
-
   char *user = getUsername(uid);
-  if (NULL == user)
-    return NULL;
-  int numgroups = 0;
-  char **groups = getGroups(uid, &numgroups);
-  hdfsFS fs = hdfsConnectAsUser(hostname, port, user);
-  freeGroups(groups, numgroups);
-  if (user) 
+  int ret;
+  hdfsFS fs = NULL;
+  if (NULL == user) {
+    goto done;
+  }
+
+  ret = pthread_mutex_lock(&tableMutex);
+  assert(0 == ret);
+
+  fs = findFs(user);
+  if (NULL == fs) {
+    fs = hdfsConnectAsUserNewInstance(hostname, port, user);
+    if (NULL == fs) {
+      ERROR("Unable to create fs for user %s", user);
+      goto done;
+    }
+    if (-1 == insertFs(user, fs)) {
+      ERROR("Unable to cache fs for user %s", user);
+    }
+  }
+
+done:
+  ret = pthread_mutex_unlock(&tableMutex);
+  assert(0 == ret);
+  if (user) {
     free(user);
+  }
   return fs;
 }
 
-#else
-
-hdfsFS doConnectAsUser(const char *hostname, int port) {
-  return hdfsConnect(hostname, port);
+/*
+ * We currently cache a fs handle per-user in this module rather
+ * than use the FileSystem cache in the java client. Therefore
+ * we do not disconnect the fs handle here.
+ */
+int doDisconnect(hdfsFS fs) {
+  return 0;
 }
-
-#endif

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.h
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.h?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.h (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_connect.h Mon Jun 20 16:03:11 2011
@@ -21,12 +21,8 @@
 
 #include "fuse_dfs.h"
 
-/**
- * Connects to the NN as the current user/group according to FUSE.
- * Uses the fuse context to get the user name and groups.
- * (if hadoop pre 0.19, will ignore user and group).
- */
-
-hdfsFS doConnectAsUser(const char *hostname, int port) ;
+hdfsFS doConnectAsUser(const char *hostname, int port);
+int doDisconnect(hdfsFS fs);
+int allocFsTable(void);
 
 #endif

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_context_handle.h
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_context_handle.h?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_context_handle.h (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_context_handle.h Mon Jun 20 16:03:11 2011
@@ -33,7 +33,6 @@ typedef struct dfs_context_struct {
   int debug;
   char *nn_hostname;
   int nn_port;
-  hdfsFS fs;
   int read_only;
   int usetrash;
   int direct_io;

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c Mon Jun 20 16:03:11 2011
@@ -20,7 +20,7 @@
 #include "fuse_options.h"
 #include "fuse_impls.h"
 #include "fuse_init.h"
-
+#include "fuse_connect.h"
 
 int is_protected(const char *path) {
 
@@ -38,55 +38,49 @@ int is_protected(const char *path) {
 }
 
 static struct fuse_operations dfs_oper = {
-  .getattr	= dfs_getattr,
-  .access	= dfs_access,
-  .readdir	= dfs_readdir,
-  .destroy       = dfs_destroy,
-  .init         = dfs_init,
-  .open	        = dfs_open,
-  .read	        = dfs_read,
-  .symlink	= dfs_symlink,
-  .statfs	= dfs_statfs,
-  .mkdir	= dfs_mkdir,
-  .rmdir	= dfs_rmdir,
-  .rename	= dfs_rename,
-  .unlink       = dfs_unlink,
-  .release      = dfs_release,
-  .create       = dfs_create,
-  .write	= dfs_write,
-  .flush        = dfs_flush,
-  .mknod        = dfs_mknod,
-  .utimens      = dfs_utimens,
-  .chmod	= dfs_chmod,
-  .chown	= dfs_chown,
-  .truncate	= dfs_truncate,
+  .getattr  = dfs_getattr,
+  .access   = dfs_access,
+  .readdir  = dfs_readdir,
+  .destroy  = dfs_destroy,
+  .init     = dfs_init,
+  .open     = dfs_open,
+  .read     = dfs_read,
+  .symlink  = dfs_symlink,
+  .statfs   = dfs_statfs,
+  .mkdir    = dfs_mkdir,
+  .rmdir    = dfs_rmdir,
+  .rename   = dfs_rename,
+  .unlink   = dfs_unlink,
+  .release  = dfs_release,
+  .create   = dfs_create,
+  .write    = dfs_write,
+  .flush    = dfs_flush,
+  .mknod    = dfs_mknod,
+  .utimens  = dfs_utimens,
+  .chmod    = dfs_chmod,
+  .chown    = dfs_chown,
+  .truncate = dfs_truncate,
 };
 
-
 int main(int argc, char *argv[])
 {
-
   umask(0);
 
   extern const char *program;  
   program = argv[0];
   struct fuse_args args = FUSE_ARGS_INIT(argc, argv);
 
-  /* clear structure that holds our options */
   memset(&options, 0, sizeof(struct options));
 
-  // some defaults
   options.rdbuffer_size = 10*1024*1024; 
   options.attribute_timeout = 60; 
   options.entry_timeout = 60;
 
-  if (fuse_opt_parse(&args, &options, dfs_opts, dfs_options) == -1)
-    /** error parsing options */
+  if (-1 == fuse_opt_parse(&args, &options, dfs_opts, dfs_options)) {
     return -1;
+  }
 
-
-  // Some fuse options we set
-  if (! options.private) {
+  if (!options.private) {
     fuse_opt_add_arg(&args, "-oallow_other");
   }
 
@@ -109,28 +103,24 @@ int main(int argc, char *argv[])
     exit(0);
   }
 
-
-  // 
-  // Check we can connect to hdfs
-  // 
+  // Check connection as root
   if (options.initchecks == 1) {
-    hdfsFS temp;
-    if ((temp = hdfsConnect(options.server, options.port)) == NULL) {
+    hdfsFS tempFS = hdfsConnectAsUser(options.server, options.port, "root");
+    if (NULL == tempFS) {
       const char *cp = getenv("CLASSPATH");
       const char *ld = getenv("LD_LIBRARY_PATH");
       ERROR("FATAL: misconfiguration - cannot connect to HDFS");
       ERROR("LD_LIBRARY_PATH=%s",ld == NULL ? "NULL" : ld);
       ERROR("CLASSPATH=%s",cp == NULL ? "NULL" : cp);
-      exit(0);
+      exit(1);
+    }
+    if (doDisconnect(tempFS)) {
+      ERROR("FATAL: unable to disconnect from test filesystem.");
+      exit(1);
     }
   }
 
   int ret = fuse_main(args.argc, args.argv, &dfs_oper, NULL);
-
-  if (ret) printf("\n");
-
-  /** free arguments */
   fuse_opt_free_args(&args);
-
   return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.h
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.h?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.h (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs.h Mon Jun 20 16:03:11 2011
@@ -35,11 +35,6 @@
 #include <config.h>
 #endif
 
-#ifdef linux
-/* For pread()/pwrite() */
-#define _XOPEN_SOURCE 500
-#endif
-
 #ifdef HAVE_SETXATTR
 #include <sys/xattr.h>
 #endif

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh Mon Jun 20 16:03:11 2011
@@ -29,7 +29,7 @@ export  JAVA_HOME=/usr/local/java
 fi
 
 if [ "$LD_LIBRARY_PATH" = "" ]; then
-export LD_LIBRARY_PATH=$JAVA_HOME/jre/lib/$OS_ARCH/server:/usr/local/share/hdfs/libhdfs/:/usr/local/lib
+export LD_LIBRARY_PATH=$JAVA_HOME/jre/lib/$OS_ARCH/server:/usr/local/lib
 fi
 
 # If dev build set paths accordingly
@@ -38,7 +38,7 @@ if [ -d $HADOOP_PREFIX/build ]; then
   for f in ${HADOOP_PREFIX}/build/*.jar ; do
     export CLASSPATH=$CLASSPATH:$f
   done
-  for f in $HADOOP_PREFIX/build/ivy/lib/Hadoop-Hdfs/common/*.jar ; do
+  for f in $HADOOP_PREFIX/build/ivy/lib/hadoop-hdfs/common/*.jar ; do
     export CLASSPATH=$CLASSPATH:$f
   done
   export PATH=$HADOOP_PREFIX/build/contrib/fuse-dfs:$PATH

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_access.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_access.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_access.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_access.c Mon Jun 20 16:03:11 2011
@@ -23,21 +23,7 @@
 int dfs_access(const char *path, int mask)
 {
   TRACE1("access", path)
-  // bugbug - I think we need the FileSystemAPI/libhdfs to expose this!
-  // retrieve dfs specific data
-  dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
-
-  // check params and the context var
-  assert(dfs);
-  assert(path);
-
-  hdfsFS userFS;
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port)) == NULL) {
-    ERROR("Could not connect to HDFS");
-    return -EIO;
-  }
-  //  return hdfsAccess(userFS, path, mask);
+  assert(path != NULL);
+  // TODO: HDFS-428
   return 0;
 }
-
-

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chmod.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chmod.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chmod.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chmod.c Mon Jun 20 16:03:11 2011
@@ -24,27 +24,30 @@
 int dfs_chmod(const char *path, mode_t mode)
 {
   TRACE1("chmod", path)
-
-#if PERMS
-  // retrieve dfs specific data
+  int ret = 0;
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect to HDFS");
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
   if (hdfsChmod(userFS, path, (short)mode)) {
     ERROR("Could not chmod %s to %d", path, (int)mode);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
-#endif
-  return 0;
+
+  return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c Mon Jun 20 16:03:11 2011
@@ -26,10 +26,9 @@
   TRACE1("chown", path)
 
   int ret = 0;
-
-#if PERMS
   char *user = NULL;
   char *group = NULL;
+  hdfsFS userFS = NULL;
 
   // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
@@ -43,36 +42,40 @@
   if (NULL == user) {
     ERROR("Could not lookup the user id string %d",(int)uid); 
     ret = -EIO;
+    goto cleanup;
   }
 
-  if (0 == ret) {
-    group = getGroup(gid);
-    if (group == NULL) {
-      ERROR("Could not lookup the group id string %d",(int)gid);
-      ret = -EIO;
-    } 
+  group = getGroup(gid);
+  if (group == NULL) {
+    ERROR("Could not lookup the group id string %d",(int)gid);
+    ret = -EIO;
+    goto cleanup;
+  } 
+
+  userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
+    ERROR("Could not connect to HDFS");
+    ret = -EIO;
+    goto cleanup;
   }
 
-  hdfsFS userFS = NULL;
-  if (0 == ret) {
-    // if not connected, try to connect and fail out if we can't.
-    if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
-      ERROR("Could not connect to HDFS");
-      ret = -EIO;
-    }
+  if (hdfsChown(userFS, path, user, group)) {
+    ERROR("Could not chown %s to %d:%d", path, (int)uid, gid);
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
   }
 
-  if (0 == ret) {
-    if (hdfsChown(userFS, path, user, group)) {
-      ERROR("Could not chown %s to %d:%d", path, (int)uid, gid);
-      ret = -EIO;
-    }
+cleanup:
+  if (userFS && doDisconnect(userFS)) {
+    ret = -EIO;
   }
-  if (user) 
+  if (user) {
     free(user);
-  if (group)
+  }
+  if (group) {
     free(group);
-#endif
+  }
+
   return ret;
 
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_getattr.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_getattr.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_getattr.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_getattr.c Mon Jun 20 16:03:11 2011
@@ -19,39 +19,36 @@
 #include "fuse_dfs.h"
 #include "fuse_impls.h"
 #include "fuse_stat_struct.h"
+#include "fuse_connect.h"
 
 int dfs_getattr(const char *path, struct stat *st)
 {
   TRACE1("getattr", path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(dfs);
   assert(path);
   assert(st);
 
-  // if not connected, try to connect and fail out if we can't.
-  if (NULL == dfs->fs && 
-      NULL == (dfs->fs = hdfsConnect(dfs->nn_hostname,dfs->nn_port))) {
+  hdfsFS fs = doConnectAsUser(dfs->nn_hostname,dfs->nn_port);
+  if (NULL == fs) {
     ERROR("Could not connect to %s:%d", dfs->nn_hostname, dfs->nn_port);
     return -EIO;
   }
 
-  // call the dfs API to get the actual information
-  hdfsFileInfo *info = hdfsGetPathInfo(dfs->fs,path);
-
+  int ret = 0;
+  hdfsFileInfo *info = hdfsGetPathInfo(fs,path);
   if (NULL == info) {
-    return -ENOENT;
+    ret = -ENOENT;
+    goto cleanup;
   }
-
   fill_stat_structure(&info[0], st);
 
   // setup hard link info - for a file it is 1 else num entries in a dir + 2 (for . and ..)
   if (info[0].mKind == kObjectKindDirectory) {
     int numEntries = 0;
-    hdfsFileInfo *info = hdfsListDirectory(dfs->fs,path,&numEntries);
+    hdfsFileInfo *info = hdfsListDirectory(fs,path,&numEntries);
 
     if (info) {
       hdfsFreeFileInfo(info,numEntries);
@@ -65,5 +62,10 @@ int dfs_getattr(const char *path, struct
   // free the info pointer
   hdfsFreeFileInfo(info,1);
 
-  return 0;
+cleanup:
+  if (doDisconnect(fs)) {
+    ERROR("Could not disconnect from filesystem");
+    ret = -EIO;
+  }
+  return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_mkdir.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_mkdir.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_mkdir.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_mkdir.c Mon Jun 20 16:03:11 2011
@@ -25,10 +25,8 @@ int dfs_mkdir(const char *path, mode_t m
 {
   TRACE1("mkdir", path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
@@ -43,25 +41,29 @@ int dfs_mkdir(const char *path, mode_t m
     return -EACCES;
   }
   
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
   // In theory the create and chmod should be atomic.
 
+  int ret = 0;
   if (hdfsCreateDirectory(userFS, path)) {
     ERROR("HDFS could not create directory %s", path);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
   }
 
-#if PERMS
   if (hdfsChmod(userFS, path, (short)mode)) {
     ERROR("Could not chmod %s to %d", path, (int)mode);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
-#endif
-  return 0;
+  return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_open.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_open.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_open.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_open.c Mon Jun 20 16:03:11 2011
@@ -39,19 +39,20 @@ int dfs_open(const char *path, struct fu
   int flags = (fi->flags & 0x7FFF);
 
   // retrieve dfs specific data
-  dfs_fh *fh = (dfs_fh*)malloc(sizeof (dfs_fh));
+  dfs_fh *fh = (dfs_fh*)calloc(1, sizeof (dfs_fh));
   if (fh == NULL) {
     ERROR("Malloc of new file handle failed");
     return -EIO;
   }
 
-  if ((fh->fs = doConnectAsUser(dfs->nn_hostname,dfs->nn_port)) == NULL) {
+  fh->fs = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (fh->fs == NULL) {
     ERROR("Could not connect to dfs");
     return -EIO;
   }
 
   if (flags & O_RDWR) {
-    hdfsFileInfo *info = hdfsGetPathInfo(dfs->fs,path);
+    hdfsFileInfo *info = hdfsGetPathInfo(fh->fs,path);
     if (info == NULL) {
       // File does not exist (maybe?); interpret it as a O_WRONLY
       // If the actual error was something else, we'll get it again when
@@ -73,28 +74,20 @@ int dfs_open(const char *path, struct fu
     return -errno;
   }
 
-  // 
-  // mutex needed for reads/writes
-  //
   pthread_mutex_init(&fh->mutex, NULL);
 
   if (fi->flags & O_WRONLY || fi->flags & O_CREAT) {
-    // write specific initialization
     fh->buf = NULL;
   } else  {
-    // read specific initialization
-
     assert(dfs->rdbuffer_size > 0);
-
-    if (NULL == (fh->buf = (char*)malloc(dfs->rdbuffer_size*sizeof (char)))) {
+    fh->buf = (char*)malloc(dfs->rdbuffer_size * sizeof(char));
+    if (NULL == fh->buf) {
       ERROR("Could not allocate memory for a read for file %s\n", path);
       ret = -EIO;
     }
-
     fh->buffersStartOffset = 0;
     fh->bufferSize = 0;
   }
-
   fi->fh = (uint64_t)fh;
 
   return ret;

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_readdir.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_readdir.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_readdir.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_readdir.c Mon Jun 20 16:03:11 2011
@@ -24,24 +24,15 @@
 int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
                        off_t offset, struct fuse_file_info *fi)
 {
-  TRACE1("readdir",path)
-
-  (void) offset;
-  (void) fi;
-
-  // retrieve dfs specific data
+  TRACE1("readdir", path)
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(dfs);
   assert(path);
   assert(buf);
 
-  int path_len = strlen(path);
-
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
@@ -50,16 +41,16 @@ int dfs_readdir(const char *path, void *
   // component (HDFS-975) would save us from parsing it out below.
   int numEntries = 0;
   hdfsFileInfo *info = hdfsListDirectory(userFS, path, &numEntries);
-  userFS = NULL;
 
+  int ret = 0;
   // NULL means either the directory doesn't exist or maybe IO error.
   if (NULL == info) {
-    return -ENOENT;
+    ret = (errno > 0) ? -errno : -ENOENT;
+    goto cleanup;
   }
 
   int i ;
   for (i = 0; i < numEntries; i++) {
-
     if (NULL == info[i].mName) {
       ERROR("Path %s info[%d].mName is NULL", path, i);
       continue;
@@ -115,5 +106,11 @@ int dfs_readdir(const char *path, void *
     }
   // free the info pointers
   hdfsFreeFileInfo(info,numEntries);
-  return 0;
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
+    ERROR("Failed to disconnect %d", errno);
+  }
+  return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c Mon Jun 20 16:03:11 2011
@@ -19,6 +19,7 @@
 #include "fuse_dfs.h"
 #include "fuse_impls.h"
 #include "fuse_file_handle.h"
+#include "fuse_connect.h"
 
 /**
  * This mutex is to protect releasing a file handle in case the user calls close in different threads
@@ -63,6 +64,11 @@ int dfs_release (const char *path, struc
     if (fh->buf != NULL) {
       free(fh->buf);
     }
+
+    if (doDisconnect(fh->fs)) {
+      ret = -EIO;
+    }
+
     // this is always created and initialized, so always destroy it. (see dfs_open)
     pthread_mutex_destroy(&fh->mutex);
 

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rename.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rename.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rename.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rename.c Mon Jun 20 16:03:11 2011
@@ -46,17 +46,23 @@ int dfs_rename(const char *from, const c
     return -EACCES;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
+  int ret = 0;
   if (hdfsRename(userFS, from, to)) {
     ERROR("Rename %s to %s failed", from, to);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
+  return ret;
 
-  return 0;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c Mon Jun 20 16:03:11 2011
@@ -27,10 +27,8 @@ int dfs_rmdir(const char *path)
 {
   TRACE1("rmdir", path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
@@ -45,27 +43,34 @@ int dfs_rmdir(const char *path)
     return -EACCES;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
+  int ret = 0;
   int numEntries = 0;
   hdfsFileInfo *info = hdfsListDirectory(userFS,path,&numEntries);
 
-  // free the info pointers
-  hdfsFreeFileInfo(info,numEntries);
+  if (info) {
+    hdfsFreeFileInfo(info, numEntries);
+  }
 
   if (numEntries) {
-    return -ENOTEMPTY;
+    ret = -ENOTEMPTY;
+    goto cleanup;
   }
 
   if (hdfsDeleteWithTrash(userFS, path, dfs->usetrash)) {
     ERROR("Error trying to delete directory %s", path);
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
-  return 0;
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
+  }
+  return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_statfs.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_statfs.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_statfs.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_statfs.c Mon Jun 20 16:03:11 2011
@@ -25,20 +25,16 @@ int dfs_statfs(const char *path, struct 
 {
   TRACE1("statfs",path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(st);
   assert(dfs);
 
-  // init the stat structure
   memset(st,0,sizeof(struct statvfs));
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
@@ -47,32 +43,15 @@ int dfs_statfs(const char *path, struct 
   const tOffset used  = hdfsGetUsed(userFS);
   const tOffset bsize = hdfsGetDefaultBlockSize(userFS);
 
-  // fill in the statvfs structure
-
-  /* FOR REFERENCE:
-     struct statvfs {
-     unsigned long  f_bsize;    // file system block size
-     unsigned long  f_frsize;   // fragment size
-     fsblkcnt_t     f_blocks;   // size of fs in f_frsize units
-     fsblkcnt_t     f_bfree;    // # free blocks
-     fsblkcnt_t     f_bavail;   // # free blocks for non-root
-     fsfilcnt_t     f_files;    // # inodes
-     fsfilcnt_t     f_ffree;    // # free inodes
-     fsfilcnt_t     f_favail;   // # free inodes for non-root
-     unsigned long  f_fsid;     // file system id
-     unsigned long  f_flag;     / mount flags
-     unsigned long  f_namemax;  // maximum filename length
-     };
-  */
+  if (doDisconnect(userFS)) {
+    return -EIO;
+  }
 
   st->f_bsize   =  bsize;
   st->f_frsize  =  bsize;
-
   st->f_blocks  =  cap/bsize;
-
   st->f_bfree   =  (cap-used)/bsize;
   st->f_bavail  =  (cap-used)/bsize;
-
   st->f_files   =  1000;
   st->f_ffree   =  500;
   st->f_favail  =  500;
@@ -82,4 +61,3 @@ int dfs_statfs(const char *path, struct 
 
   return 0;
 }
-

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_truncate.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_truncate.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_truncate.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_truncate.c Mon Jun 20 16:03:11 2011
@@ -29,9 +29,6 @@
 int dfs_truncate(const char *path, off_t size)
 {
   TRACE1("truncate", path)
-  if (size != 0) {
-    return -ENOTSUP;
-  }
 
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
@@ -39,16 +36,20 @@ int dfs_truncate(const char *path, off_t
   assert('/' == *path);
   assert(dfs);
 
+  if (size != 0) {
+    return -ENOTSUP;
+  }
+
   int ret = dfs_unlink(path);
   if (ret != 0) {
     return ret;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port)) == NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
   int flags = O_WRONLY | O_CREAT;
@@ -56,12 +57,19 @@ int dfs_truncate(const char *path, off_t
   hdfsFile file;
   if ((file = (hdfsFile)hdfsOpenFile(userFS, path, flags,  0, 0, 0)) == NULL) {
     ERROR("Could not connect open file %s", path);
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
   if (hdfsCloseFile(userFS, file) != 0) {
     ERROR("Could not close file %s", path);
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
-  return 0;
+  return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_unlink.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_unlink.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_unlink.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_unlink.c Mon Jun 20 16:03:11 2011
@@ -26,16 +26,15 @@ int dfs_unlink(const char *path)
 {
   TRACE1("unlink", path)
 
-  // retrieve dfs specific data
+  int ret = 0;
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
 
   if (is_protected(path)) {
-    ERROR("Trying to delete protected directory %s ", path);
+    ERROR("Trying to delete protected directory %s", path);
     return -EACCES;
   }
 
@@ -44,17 +43,22 @@ int dfs_unlink(const char *path)
     return -EACCES;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
   if (hdfsDeleteWithTrash(userFS, path, dfs->usetrash)) {
     ERROR("Could not delete file %s", path);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
+  return ret;
 
-  return 0;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_utimens.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_utimens.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_utimens.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_impls_utimens.c Mon Jun 20 16:03:11 2011
@@ -23,11 +23,9 @@
 int dfs_utimens(const char *path, const struct timespec ts[2])
 {
   TRACE1("utimens", path)
-#if PERMS
-  // retrieve dfs specific data
+  int ret = 0;
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
@@ -35,25 +33,31 @@ int dfs_utimens(const char *path, const 
   time_t aTime = ts[0].tv_sec;
   time_t mTime = ts[1].tv_sec;
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
   if (hdfsUtime(userFS, path, mTime, aTime)) {
-    hdfsFileInfo *info = hdfsGetPathInfo(dfs->fs,path);
+    hdfsFileInfo *info = hdfsGetPathInfo(userFS, path);
     if (info == NULL) {
-      return -EIO;
+      ret = (errno > 0) ? -errno : -ENOENT;
+      goto cleanup;
     }
     // Silently ignore utimens failure for directories, otherwise 
     // some programs like tar will fail.
     if (info->mKind == kObjectKindDirectory) {
-      return 0;
+      ret = 0;
+    } else {
+      ret = (errno > 0) ? -errno : -EACCES;
     }
-    return -errno;
+    goto cleanup;
   }
-#endif  
-  return 0;
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
+  }
+  return ret;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_init.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_init.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_init.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_init.c Mon Jun 20 16:03:11 2011
@@ -22,6 +22,7 @@
 #include "fuse_init.h"
 #include "fuse_options.h"
 #include "fuse_context_handle.h"
+#include "fuse_connect.h"
 
 // Hacked up function to basically do:
 //  protectedpaths = split(options.protected,':');
@@ -30,7 +31,6 @@ void init_protectedpaths(dfs_context *df
 
   char *tmp = options.protected;
 
-
   // handle degenerate case up front.
   if (tmp == NULL || 0 == *tmp) {
     dfs->protectedpaths = (char**)malloc(sizeof(char*));
@@ -43,7 +43,6 @@ void init_protectedpaths(dfs_context *df
     print_options();
   }
 
-
   int i = 0;
   while (tmp && (NULL != (tmp = index(tmp,':')))) {
     tmp++; // pass the ,
@@ -75,27 +74,15 @@ void init_protectedpaths(dfs_context *df
     j++;
   }
   dfs->protectedpaths[j] = NULL;
-
-  /*
-    j  = 0;
-    while (dfs->protectedpaths[j]) {
-    printf("dfs->protectedpaths[%d]=%s\n",j,dfs->protectedpaths[j]);
-    fflush(stdout);
-    j++;
-    }
-    exit(1);
-  */
 }
 
-void *dfs_init()
-{
 
+void *dfs_init(void) {
   //
   // Create a private struct of data we will pass to fuse here and which
   // will then be accessible on every call.
   //
-  dfs_context *dfs = (dfs_context*)malloc(sizeof (dfs_context));
-
+  dfs_context *dfs = (dfs_context*)malloc(sizeof(dfs_context));
   if (NULL == dfs) {
     ERROR("FATAL: could not malloc dfs_context");
     exit(1);
@@ -105,7 +92,6 @@ void *dfs_init()
   dfs->debug                 = options.debug;
   dfs->nn_hostname           = options.server;
   dfs->nn_port               = options.port;
-  dfs->fs                    = NULL;
   dfs->read_only             = options.read_only;
   dfs->usetrash              = options.usetrash;
   dfs->protectedpaths        = NULL;
@@ -121,14 +107,17 @@ void *dfs_init()
     DEBUG("dfs->rdbuffersize <= 0 = %ld", dfs->rdbuffer_size);
     dfs->rdbuffer_size = 32768;
   }
+
+  if (0 != allocFsTable()) {
+    ERROR("FATAL: could not allocate ");
+    exit(1);
+  }
+
   return (void*)dfs;
 }
 
 
-
-void dfs_destroy (void *ptr)
+void dfs_destroy(void *ptr)
 {
   TRACE("destroy")
-  dfs_context *dfs = (dfs_context*)ptr;
-  dfs->fs = NULL;
 }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_options.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_options.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_options.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_options.c Mon Jun 20 16:03:11 2011
@@ -107,10 +107,10 @@ int dfs_options(void *data, const char *
 
   switch (key) {
   case FUSE_OPT_KEY_OPT:
-    fprintf(stderr, "fuse-dfs ignoring option %s\n", arg);
+    INFO("Ignoring option %s", arg);
     return 1;
-  case  KEY_VERSION:
-    fprintf(stdout, "%s %s\n", program, _FUSE_DFS_VERSION);
+  case KEY_VERSION:
+    INFO("%s %s\n", program, _FUSE_DFS_VERSION);
     exit(0);
   case KEY_HELP:
     print_usage(program);
@@ -159,14 +159,13 @@ int dfs_options(void *data, const char *
       } else if (strcmp(arg,"rw") == 0) {
         options.read_only = 0;
       } else {
-        ERROR("fuse-dfs didn't recognize %s,%d\n",arg,key);
-        fuse_opt_add_arg(outargs,arg);
+        INFO("Adding FUSE arg %s", arg);
+        fuse_opt_add_arg(outargs, arg);
         return 0;
       }
     } else {
       options.port = tmp_port;
       options.server = strdup(tmp_server);
-      ERROR("port=%d,server=%s\n", options.port, options.server);
     }
   }
   }

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_stat_struct.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_stat_struct.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_stat_struct.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_stat_struct.c Mon Jun 20 16:03:11 2011
@@ -25,18 +25,15 @@
 #include "fuse_stat_struct.h"
 #include "fuse_context_handle.h"
 
-#if PERMS
-/**
+/*
  * getpwuid and getgrgid return static structs so we safeguard the contents
  * while retrieving fields using the 2 structs below.
  * NOTE: if using both, always get the passwd struct firt!
  */
 extern pthread_mutex_t passwdstruct_mutex; 
 extern pthread_mutex_t groupstruct_mutex;
-#endif
-
 
-const int default_id       = 99; // nobody  - not configurable since soon uids in dfs, yeah!
+const int default_id = 99; // nobody  - not configurable since soon uids in dfs, yeah!
 const int blksize = 512;
 
 /**
@@ -55,7 +52,6 @@ int fill_stat_structure(hdfsFileInfo *in
   st->st_nlink = (info->mKind == kObjectKindDirectory) ? 0 : 1;
 
   uid_t owner_id = default_id;
-#if PERMS
   if (info->mOwner != NULL) {
     //
     // Critical section - protect from concurrent calls in different threads since
@@ -73,9 +69,9 @@ int fill_stat_structure(hdfsFileInfo *in
     pthread_mutex_unlock(&passwdstruct_mutex);
 
   } 
-#endif
+
   gid_t group_id = default_id;
-#if PERMS
+
   if (info->mGroup != NULL) {
     //
     // Critical section - protect from concurrent calls in different threads since
@@ -93,15 +89,12 @@ int fill_stat_structure(hdfsFileInfo *in
     pthread_mutex_unlock(&groupstruct_mutex);
 
   }
-#endif
 
   short perm = (info->mKind == kObjectKindDirectory) ? (S_IFDIR | 0777) :  (S_IFREG | 0666);
-#if PERMS
   if (info->mPermissions > 0) {
     perm = (info->mKind == kObjectKindDirectory) ? S_IFDIR:  S_IFREG ;
     perm |= info->mPermissions;
   }
-#endif
 
   // set stat metadata
   st->st_size     = (info->mKind == kObjectKindDirectory) ? 4096 : info->mSize;
@@ -110,11 +103,7 @@ int fill_stat_structure(hdfsFileInfo *in
   st->st_mode     = perm;
   st->st_uid      = owner_id;
   st->st_gid      = group_id;
-#if PERMS
   st->st_atime    = info->mLastAccess;
-#else
-  st->st_atime    = info->mLastMod;
-#endif
   st->st_mtime    = info->mLastMod;
   st->st_ctime    = info->mLastMod;
 

Modified: hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_users.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_users.c?rev=1137675&r1=1137674&r2=1137675&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_users.c (original)
+++ hadoop/common/trunk/hdfs/src/contrib/fuse-dfs/src/fuse_users.c Mon Jun 20 16:03:11 2011
@@ -24,26 +24,19 @@
 
 #include "fuse_dfs.h"
 
-
-
-#if PERMS
-/**
+/*
  * getpwuid and getgrgid return static structs so we safeguard the contents
  * while retrieving fields using the 2 structs below.
  * NOTE: if using both, always get the passwd struct firt!
  */
 pthread_mutex_t passwdstruct_mutex = PTHREAD_MUTEX_INITIALIZER;
 pthread_mutex_t groupstruct_mutex = PTHREAD_MUTEX_INITIALIZER;
-#endif
 
-#if PERMS
-
-/**
+/*
  * Utility for getting the user making the fuse call in char * form
  * NOTE: if non-null return, the return must be freed by the caller.
  */
-char *getUsername(uid_t uid)
-{
+char *getUsername(uid_t uid) {
   //
   // Critical section - protect from concurrent calls in different threads.
   // since the struct below is static.
@@ -218,4 +211,3 @@ char ** getGroups(uid_t uid, int *num_gr
 #endif
   return groupnames;
 }
-#endif