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 zs...@apache.org on 2008/09/18 21:34:29 UTC

svn commit: r696777 - in /hadoop/core/trunk: CHANGES.txt src/contrib/fuse-dfs/README src/contrib/fuse-dfs/configure.ac src/contrib/fuse-dfs/src/fuse_dfs.c src/contrib/fuse-dfs/src/test/TestFuseDFS.java

Author: zshao
Date: Thu Sep 18 12:34:28 2008
New Revision: 696777

URL: http://svn.apache.org/viewvc?rev=696777&view=rev
Log:
HADOOP-4106. libhdfs: add time, permission and user attribute support (part 2). (Pete Wyckoff through zshao)

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/contrib/fuse-dfs/README
    hadoop/core/trunk/src/contrib/fuse-dfs/configure.ac
    hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_dfs.c
    hadoop/core/trunk/src/contrib/fuse-dfs/src/test/TestFuseDFS.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=696777&r1=696776&r2=696777&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Thu Sep 18 12:34:28 2008
@@ -180,6 +180,9 @@
 
   IMPROVEMENTS
 
+    HADOOP-4106. libhdfs: add time, permission and user attribute support (part 2).
+    (Pete Wyckoff through zshao)
+
     HADOOP-4104. libhdfs: add time, permission and user attribute support.
     (Pete Wyckoff through zshao)
 

Modified: hadoop/core/trunk/src/contrib/fuse-dfs/README
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/README?rev=696777&r1=696776&r2=696777&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/README (original)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/README Thu Sep 18 12:34:28 2008
@@ -62,7 +62,7 @@
 
 1. add the following to /etc/fstab
 
-fuse_dfs#dfs://hadoop_server.foo.com:9000 /export/hdfs fuse -oallow_other,rw,-ousetrash 0 0
+fuse_dfs#dfs://hadoop_server.foo.com:9000 /export/hdfs fuse -oallow_other,rw,-ousetrash,-oinitchecks 0 0
 
 
 2. Mount using: `mount /export/hdfs`. Expect problems with not finding fuse_dfs. You will need to probably add this to /sbin and then problems finding the above 3 libraries. Add these using ldconfig.
@@ -83,7 +83,7 @@
 -onotrash (opposite of usetrash)
 -odebug (do not daemonize - aka -d in fuse speak)
 -obig_writes (use fuse big_writes option so as to allow better performance of writes on kernels >= 2.6.26)
-
+-initchecks - have fuse-dfs try to connect to hdfs to ensure all is ok upon startup. recommended to have this  on
 The defaults are:
 
 entry,attribute_timeouts = 60 seconds
@@ -111,6 +111,7 @@
 
 3. use -obig_writes
 
+4. use -initchecks
 
 KNOWN ISSUES 
 

Modified: hadoop/core/trunk/src/contrib/fuse-dfs/configure.ac
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/configure.ac?rev=696777&r1=696776&r2=696777&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/configure.ac (original)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/configure.ac Thu Sep 18 12:34:28 2008
@@ -29,6 +29,7 @@
 AC_PREFIX_DEFAULT([`pwd`])
 
 
+
 #case $target in
 #*64*intel)
 #     OS_ARCH=intel64 ;;
@@ -39,6 +40,9 @@
  DEFS=""
 AC_SUBST([DEFS])
 
+AC_FUNC_GETGROUPS
+AC_TYPE_GETGROUPS
+
 ############################################################################
 # Section 2:
 # User Configurable system defaults. Change With CAUTION!

Modified: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_dfs.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_dfs.c?rev=696777&r1=696776&r2=696777&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_dfs.c (original)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_dfs.c Thu Sep 18 12:34:28 2008
@@ -47,6 +47,9 @@
 
 #include <hdfs.h>
 #include <stddef.h>
+#include <sys/types.h>
+#include <grp.h>
+#include <pwd.h>
 
 // Constants
 //
@@ -54,7 +57,6 @@
 static const int blksize = 512;
 static const char *const TrashPrefixDir = "/Trash";
 static const char *const TrashDir = "/Trash/Current";
-#define OPTIMIZED_READS 1
 static const char *program;
 
 
@@ -65,6 +67,7 @@
   int port;
   int debug;
   int read_only;
+  int initchecks;
   int usetrash;
   int entry_timeout;
   int attribute_timeout;
@@ -92,6 +95,7 @@
   char *buf;
   tSize sizeBuffer;  //what is the size of the buffer we have
   off_t startOffset; //where the buffer starts in the file
+  hdfsFS fs; // for writes need to access as the real user
 } dfs_fh;
 
 
@@ -117,6 +121,7 @@
     KEY_PRIVATE,
     KEY_BIGWRITES,
     KEY_DEBUG,
+    KEY_INITCHECKS,
   };
 
 static struct fuse_opt dfs_opts[] =
@@ -131,6 +136,7 @@
     FUSE_OPT_KEY("private", KEY_PRIVATE),
     FUSE_OPT_KEY("ro", KEY_RO),
     FUSE_OPT_KEY("debug", KEY_DEBUG),
+    FUSE_OPT_KEY("initchecks", KEY_INITCHECKS),
     FUSE_OPT_KEY("big_writes", KEY_BIGWRITES),
     FUSE_OPT_KEY("rw", KEY_RW),
     FUSE_OPT_KEY("usetrash", KEY_USETRASH),
@@ -177,6 +183,9 @@
     fuse_opt_add_arg(outargs, "-d");
     options.debug = 1;
     break;
+  case KEY_INITCHECKS:
+    options.initchecks = 1;
+    break;
   case KEY_BIGWRITES:
 #ifdef FUSE_CAP_BIG_WRITES
     fuse_opt_add_arg(outargs, "-obig_writes");
@@ -245,7 +254,7 @@
 // NOTE: this function is a c implementation of org.apache.hadoop.fs.Trash.moveToTrash(Path path).
 //
 
-int move_to_trash(const char *item) {
+int move_to_trash(const char *item, hdfsFS userFS) {
 
   // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
@@ -256,11 +265,6 @@
   assert('/' == *item);
   assert(rindex(item,'/') >= 0);
 
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
 
   char fname[4096]; // or last element of the directory path
   char parent_directory[4096]; // the directory the fname resides in
@@ -290,7 +294,7 @@
   }
 
   // create the target trash directory in trash (if needed)
-  if ( hdfsExists(dfs->fs, trash_dir)) {
+  if ( hdfsExists(userFS, trash_dir)) {
     int status;
     // make the directory to put it in in the Trash - NOTE
     // dfs_mkdir also creates parents, so Current will be created if it does not exist.
@@ -311,7 +315,7 @@
   }
 
   // NOTE: this loop differs from the java version by capping the #of tries
-  for (j = 1; ! hdfsExists(dfs->fs, target) && j < TRASH_RENAME_TRIES ; j++) {
+  for (j = 1; ! hdfsExists(userFS, target) && j < TRASH_RENAME_TRIES ; j++) {
     if(snprintf(target, sizeof target,"%s/%s.%d",trash_dir, fname, j) >= sizeof target) {
       syslog(LOG_ERR, "move_to_trash error target is not big enough to hold new name for %s %s:%d\n",item, __FILE__, __LINE__);
       return -EIO;
@@ -322,6 +326,169 @@
 }
 
 
+/**
+ * Converts from a hdfs hdfsFileInfo to a POSIX stat struct
+ *
+ */
+int fill_stat_structure(hdfsFileInfo *info, struct stat *st) 
+{
+
+  // initialize the stat structure
+  memset(st, 0, sizeof(struct stat));
+
+  // by default: set to 0 to indicate not supported for directory because we cannot (efficiently) get this info for every subdirectory
+  st->st_nlink = (info->mKind == kObjectKindDirectory) ? 0 : 1;
+
+  uid_t owner_id = default_id;
+  if(info->mOwner != NULL) {
+    struct passwd *passwd_info = getpwnam(info->mOwner);
+    owner_id = passwd_info == NULL ? default_id : passwd_info->pw_uid;
+  } 
+
+  gid_t group_id = default_id;
+  if(info->mGroup == NULL) {
+    struct group *group_info = getgrnam(info->mGroup);
+    group_id = group_info == NULL ? default_id : group_info->gr_gid;
+  }
+
+  short perm = (info->mKind == kObjectKindDirectory) ? (S_IFDIR | 0777) :  (S_IFREG | 0666);
+  if(info->mPermissions > 0) {
+    perm = (info->mKind == kObjectKindDirectory) ? S_IFDIR:  S_IFREG ;
+    perm |= info->mPermissions;
+  }
+
+
+  // set stat metadata
+  st->st_size     = (info->mKind == kObjectKindDirectory) ? 4096 : info->mSize;
+  st->st_blksize  = blksize;
+  st->st_blocks   =  ceil(st->st_size/st->st_blksize);
+  st->st_mode     = perm;
+  st->st_uid      = owner_id;
+  st->st_gid      = group_id;
+  st->st_atime    = info->mLastMod;
+  st->st_mtime    = info->mLastMod;
+  st->st_ctime    = info->mLastMod;
+
+  return 0;
+}
+
+static char* getUsername(uid_t uid)
+{
+  struct passwd *userinfo = getpwuid(uid);
+  if(userinfo != NULL) {
+    fprintf(stderr, "DEBUG: uid=%d,%s\n",uid,userinfo->pw_name);
+    return userinfo->pw_name;
+  }
+  else
+    return NULL;
+}
+
+#define GROUPBUF_SIZE 5
+
+static void freeGroups(char **groups, int numgroups) {
+  if(groups == NULL) {
+    return;
+  }
+  int i ;
+  for(i = 0; i < numgroups; i++) {
+    free(groups[i]);
+  }
+  free(groups);
+}
+
+
+static char ** getGroups(uid_t uid, int *num_groups)
+{
+  struct passwd *userinfo = getpwuid(uid);
+
+  if (userinfo == NULL)
+    return NULL;
+  assert(userinfo->pw_name);
+
+  int user_name_len = strlen(userinfo->pw_name);
+  char **groupnames = NULL;
+
+  // see http://www.openldap.org/lists/openldap-devel/199903/msg00023.html
+#ifdef GETGROUPS_T
+  *num_groups = GROUPBUF_SIZE;
+
+  gid_t* grouplist = malloc(GROUPBUF_SIZE * sizeof(gid_t)); 
+  assert(grouplist != NULL);
+  gid_t* tmp_grouplist; 
+  int rtr;
+  if((rtr = getgrouplist(userinfo->pw_name, userinfo->pw_gid, grouplist, num_groups)) == -1) {
+    // the buffer we passed in is < *num_groups
+    if((tmp_grouplist = realloc(grouplist, *num_groups * sizeof(gid_t))) != NULL) {
+      grouplist = tmp_grouplist;
+      getgrouplist(userinfo->pw_name, userinfo->pw_gid, grouplist, num_groups);
+    }
+  }
+
+  groupnames = (char**)malloc(sizeof(char*)* (*num_groups) + 1);
+  assert(groupnames);
+  int i;
+  for(i=0; i < *num_groups; i++)
+    {
+      struct group* grp = getgrgid(grouplist[i]);
+      if (grp != NULL) {
+        int grp_name_len = strlen(grp->gr_name);
+          groupnames[i] = (char*)malloc(sizeof(char)*grp_name_len+1);
+          assert(groupnames[i] != NULL);
+          strcpy(groupnames[i], grp->gr_name);
+      } else {
+        fprintf(stderr,"Coudlnt find a group for guid %d\n", grouplist[i]);
+      }
+    }
+  free(grouplist);
+  groupnames[i] = (char*)malloc(sizeof(char)*user_name_len+1);
+  assert(groupnames[i] != NULL);
+  strcpy(groupnames[i], userinfo->pw_name);
+
+#else
+
+  struct group* grp = getgrgid( userinfo->pw_gid);
+  assert(grp->gr_name);
+  int grp_name_len = strlen(grp->gr_name);
+  groupnames = (char**)malloc(sizeof(char*)*3);
+  assert(groupnames);
+
+  int i = 0;
+  groupnames[i] = (char*)malloc(sizeof(char)*user_name_len+1);
+  assert(groupnames[i] != NULL);
+  strcpy(groupnames[i], userinfo->pw_name);
+  i++;
+
+  if(grp->grp_name != NULL) {
+    groupnames[i] = (char*)malloc(sizeof(char)*strlen(grp->grp_name)+1); \
+    assert(groupnames[i] != NULL);
+    strcpy(groupnames[i], grp->grp_name);
+  }
+  i++;
+
+  *num_groups = i;
+
+#endif
+  return groupnames;
+}
+
+/**
+ * Connects to the NN as the current user/group according to FUSE
+ *
+ */
+
+
+static hdfsFS doConnectAsUser(const char *hostname, int port) {
+  uid_t uid = fuse_get_context()->uid;
+
+  char *user = getUsername(uid);
+  int numgroups = 0;
+  char **groups = getGroups(uid, &numgroups);
+  hdfsFS fs = hdfsConnectAsUser(hostname, port, user, groups, numgroups);
+  freeGroups(groups, numgroups);
+
+  return fs;
+}
+
 
 //
 // Start of read-only functions
@@ -332,8 +499,6 @@
   // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  syslog(LOG_ERR, "starting dfs_getattr for %s\n",path);
-
   // check params and the context var
   assert(dfs);
   assert(path);
@@ -352,8 +517,7 @@
     return -ENOENT;
   }
 
-  // initialize the stat structure
-  memset(st, 0, sizeof(struct stat));
+  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) {
@@ -369,17 +533,6 @@
     st->st_nlink = 1;
   }
 
-  // set stat metadata
-  st->st_size     = (info[0].mKind == kObjectKindDirectory) ? 4096 : info[0].mSize;
-  st->st_blksize  = blksize;
-  st->st_blocks   =  ceil(st->st_size/st->st_blksize);
-  st->st_mode     = (info[0].mKind == kObjectKindDirectory) ? (S_IFDIR | 0777) :  (S_IFREG | 0666);
-  st->st_uid      = default_id;
-  st->st_gid      = default_id;
-  st->st_atime    = info[0].mLastMod;
-  st->st_mtime    = info[0].mLastMod;
-  st->st_ctime    = info[0].mLastMod;
-
   // free the info pointer
   hdfsFreeFileInfo(info,1);
 
@@ -392,8 +545,6 @@
   (void) offset;
   (void) fi;
 
-  syslog(LOG_ERR, "starting dfs_readdir for %s\n",path);
-
   // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
@@ -402,17 +553,18 @@
   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 (NULL == dfs->fs && NULL == (dfs->fs = hdfsConnect(dfs->nn_hostname,dfs->nn_port))) {
+  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
     syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
     return -EIO;
   }
 
-  int path_len = strlen(path);
-
   // call dfs to read the dir
   int numEntries = 0;
-  hdfsFileInfo *info = hdfsListDirectory(dfs->fs,path,&numEntries);
+  hdfsFileInfo *info = hdfsListDirectory(userFS,path,&numEntries);
 
   // NULL means either the directory doesn't exist or maybe IO error.
   if (NULL == info) {
@@ -429,21 +581,7 @@
     }
 
     struct stat st;
-    memset(&st, 0, sizeof(struct stat));
-
-    // set to 0 to indicate not supported for directory because we cannot (efficiently) get this info for every subdirectory
-    st.st_nlink = (info[i].mKind == kObjectKindDirectory) ? 0 : 1;
-
-    // setup stat size and acl meta data
-    st.st_size    = info[i].mSize;
-    st.st_blksize = 512;
-    st.st_blocks  =  ceil(st.st_size/st.st_blksize);
-    st.st_mode    = (info[i].mKind == kObjectKindDirectory) ? (S_IFDIR | 0777) :  (S_IFREG | 0666);
-    st.st_uid     = default_id;
-    st.st_gid     = default_id;
-    st.st_atime   = info[i].mLastMod;
-    st.st_mtime   = info[i].mLastMod;
-    st.st_ctime   = info[i].mLastMod;
+    fill_stat_structure(&info[i], &st);
 
     // hack city: todo fix the below to something nicer and more maintainable but
     // with good performance
@@ -492,8 +630,6 @@
 
   // free the info pointers
   hdfsFreeFileInfo(info,numEntries);
-  syslog(LOG_ERR, "returning dfs_readdir for %s\n",path);
-
   return 0;
 }
 
@@ -508,25 +644,15 @@
   assert(path);
   assert(buf);
 
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-
-
-#ifdef OPTIMIZED_READS
   dfs_fh *fh = (dfs_fh*)fi->fh;
   //fprintf(stderr, "Cache bounds for %s: %llu -> %llu (%d bytes). Check for offset %llu\n", path, fh->startOffset, fh->startOffset + fh->sizeBuffer, fh->sizeBuffer, offset);
   if (fh->sizeBuffer == 0  || offset < fh->startOffset || offset > (fh->startOffset + fh->sizeBuffer)  )
     {
       // do the actual read
       //fprintf (stderr,"Reading %s from HDFS, offset %llu, amount %d\n", path, offset, dfs->rdbuffer_size);
-      const tSize num_read = hdfsPread(dfs->fs, fh->hdfsFH, offset, fh->buf, dfs->rdbuffer_size);
+      const tSize num_read = hdfsPread(fh->fs, fh->hdfsFH, offset, fh->buf, dfs->rdbuffer_size);
       if (num_read < 0) {
 	syslog(LOG_ERR, "Read error - pread failed for %s with return code %d %s:%d", path, num_read, __FILE__, __LINE__);
-	hdfsDisconnect(dfs->fs);
-	dfs->fs = NULL;
 	return -EIO;
       }
       fh->sizeBuffer = num_read;
@@ -547,28 +673,6 @@
   //fprintf(stderr,"Read %s from cache, %d bytes from position %d\n", path, amount, cacheLookupOffset);
   //fprintf(stderr,"Cache status for %s: %d bytes cached from offset %llu\n", path, fh->sizeBuffer, fh->startOffset);
   return amount;
-
-#else
-  // NULL means either file doesn't exist or maybe IO error - i.e., the dfs_open must have failed
-  if (NULL == (void*)fi->fh) {
-    // should never happen
-    return  -EIO;
-  }
-  syslog(LOG_DEBUG,"buffer size=%d\n",(int)size);
-
-  // do the actual read
-  const tSize num_read = hdfsPread(dfs->fs, (hdfsFile)fi->fh, offset, buf, size);
-
-  // handle errors
-  if (num_read < 0) {
-    syslog(LOG_ERR, "Read error - pread failed for %s with return code %d %s:%d", path, num_read, __FILE__, __LINE__);
-    hdfsDisconnect(dfs->fs);
-    dfs->fs = NULL;
-    return -EIO;
-  }
-  return num_read;
-#endif
-
 }
 
 static int dfs_statfs(const char *path, struct statvfs *st)
@@ -584,15 +688,16 @@
   // 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 (NULL == dfs->fs && NULL == (dfs->fs = hdfsConnect(dfs->nn_hostname,dfs->nn_port))) {
+  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
     syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
     return -EIO;
   }
 
-  const long cap   = hdfsGetCapacity(dfs->fs);
-  const long used  = hdfsGetUsed(dfs->fs);
-  const long bsize = hdfsGetDefaultBlockSize(dfs->fs);
+  const long cap   = hdfsGetCapacity(userFS);
+  const long used  = hdfsGetUsed(userFS);
+  const long bsize = hdfsGetDefaultBlockSize(userFS);
 
   // fill in the statvfs structure
 
@@ -627,18 +732,6 @@
   return 0;
 }
 
-static int dfs_access(const char *path, int mask)
-{
-  // no permissions on dfs, always a success
-  return 0;
-}
-
-//
-// The remainder are write functionality and therefore not implemented right now
-//
-
-
-
 
 static int dfs_mkdir(const char *path, mode_t mode)
 {
@@ -648,13 +741,6 @@
   // check params and the context var
   assert(path);
   assert(dfs);
-
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-
   assert('/' == *path);
 
   int i ;
@@ -665,13 +751,19 @@
     }
   }
 
-
   if (dfs->read_only) {
     syslog(LOG_ERR,"ERROR: hdfs is configured as read-only, cannot create the directory %s\n",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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
 
-  if (hdfsCreateDirectory(dfs->fs, path)) {
+  if (hdfsCreateDirectory(userFS, path)) {
     syslog(LOG_ERR,"ERROR: hdfs trying to create directory %s",path);
     return -EIO;
   }
@@ -690,12 +782,6 @@
   assert(to);
   assert(dfs);
 
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-
   assert('/' == *from);
   assert('/' == *to);
 
@@ -716,7 +802,14 @@
     return -EACCES;
   }
 
-  if (hdfsRename(dfs->fs, from, to)) {
+  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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  if (hdfsRename(userFS, from, to)) {
     syslog(LOG_ERR,"ERROR: hdfs trying to rename %s to %s",from, to);
     return -EIO;
   }
@@ -745,13 +838,6 @@
   // check params and the context var
   assert(path);
   assert(dfs);
-
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-
   assert('/' == *path);
 
   if(is_protected(path)) {
@@ -759,8 +845,15 @@
     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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
   int numEntries = 0;
-  hdfsFileInfo *info = hdfsListDirectory(dfs->fs,path,&numEntries);
+  hdfsFileInfo *info = hdfsListDirectory(userFS,path,&numEntries);
 
   // free the info pointers
   hdfsFreeFileInfo(info,numEntries);
@@ -770,16 +863,15 @@
   }
 
   if (dfs->usetrash && strncmp(path, TrashPrefixDir, strlen(TrashPrefixDir)) != 0) {
-    return move_to_trash(path);
+    return move_to_trash(path, userFS);
   }
 
-
   if (dfs->read_only) {
     syslog(LOG_ERR,"ERROR: hdfs is configured as read-only, cannot delete the directory %s\n",path);
     return -EACCES;
   }
 
-  if(hdfsDelete(dfs->fs, path)) {
+  if(hdfsDelete(userFS, path)) {
     syslog(LOG_ERR,"ERROR: hdfs error trying to delete the directory %s\n",path);
     return -EIO;
   }
@@ -796,13 +888,6 @@
   // check params and the context var
   assert(path);
   assert(dfs);
-
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-
   assert('/' == *path);
 
   if(is_protected(path)) {
@@ -810,9 +895,16 @@
     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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
   // move the file to the trash if this is enabled and its not actually in the trash.
   if (dfs->usetrash && strncmp(path, TrashPrefixDir, strlen(TrashPrefixDir)) != 0) {
-    return move_to_trash(path);
+    return move_to_trash(path, userFS);
   }
 
   if (dfs->read_only) {
@@ -820,7 +912,7 @@
     return -EACCES;
   }
 
-  if (hdfsDelete(dfs->fs, path)) {
+  if (hdfsDelete(userFS, path)) {
     syslog(LOG_ERR,"ERROR: hdfs trying to delete the file %s",path);
     return -EIO;
   }
@@ -828,35 +920,98 @@
 
 }
 
+static int dfs_utimens(const char *path, const struct timespec ts[2])
+{
+  // 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);
+
+  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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  if (hdfsUtime(userFS, path, mTime, aTime)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to utime %s to %ld/%ld",path, (long)mTime, (long)aTime);
+    fprintf(stderr,"ERROR: could not set utime for path %s\n",path);
+    return -EIO;
+  }
+  
+  return 0;
+}
+
 static int dfs_chmod(const char *path, mode_t mode)
 {
-  (void)path;
-  (void)mode;
-  return -ENOTSUP;
+  // 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);
+
+  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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  if (hdfsChmod(userFS, path, (short)mode)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to chmod %s to %d",path, (int)mode);
+    return -EIO;
+  }
+
+  return 0;
 }
 
 static int dfs_chown(const char *path, uid_t uid, gid_t gid)
 {
-  (void)path;
-  (void)uid;
-  (void)gid;
-  return -ENOTSUP;
-}
+  // 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);
+
+  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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  char *user = getUsername(uid);
+  struct group *group_info = getgrgid(gid);
+  const char *group = group_info ? group_info->gr_name : NULL;
+  if(group_info == NULL) {
+    syslog(LOG_ERR,"Could not lookup the group id string %d\n",(int)gid); 
+    fprintf(stderr, "could not lookup group\n"); 
+  }
+
+  if (hdfsChown(userFS, path, user, group)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to chown %s to %d/%d",path, (int)uid, gid);
+    return -EIO;
+  }
+  return 0;
 
-//static int dfs_truncate(const char *path, off_t size)
-//{
-//  (void)path;
-//  (void)size;
-//  return -ENOTSUP;
-//}
+}
 
-long tempfh = 0;
 
 static int dfs_open(const char *path, struct fuse_file_info *fi)
 {
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-
   // check params and the context var
   assert(path);
   assert('/' == *path);
@@ -864,22 +1019,22 @@
 
   int ret = 0;
 
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-
   // 0x8000 is always passed in and hadoop doesn't like it, so killing it here
   // bugbug figure out what this flag is and report problem to Hadoop JIRA
   int flags = (fi->flags & 0x7FFF);
 
-
-#ifdef OPTIMIZED_READS
   // retrieve dfs specific data
   dfs_fh *fh = (dfs_fh*)malloc(sizeof (dfs_fh));
   fi->fh = (uint64_t)fh;
-  fh->hdfsFH = (hdfsFile)hdfsOpenFile(dfs->fs, path, flags,  0, 3, 0);
+
+  // if not connected, try to connect and fail out if we can't.
+  if((fh->fs = doConnectAsUser(dfs->nn_hostname,dfs->nn_port)) == NULL) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  fh->hdfsFH = (hdfsFile)hdfsOpenFile(fh->fs, path, flags,  0, 3, 0);
+
   assert(dfs->rdbuffer_size > 0);
   fh->buf = (char*)malloc(dfs->rdbuffer_size*sizeof (char));
 
@@ -890,36 +1045,6 @@
     syslog(LOG_ERR, "ERROR: could not open file %s dfs %s:%d\n", path,__FILE__, __LINE__);
     ret = -EIO;
   }
-#else
-  //  fprintf(stderr,"hdfsOpenFile being called %s,%o\n",path,flags);
-
-  // bugbug should stop O_RDWR flag here.
-
-
-  // bugbug when fix  https://issues.apache.org/jira/browse/HADOOP-3723 can remove the below code
-  if (flags & O_WRONLY) {
-    flags = O_WRONLY;
-
-  }
-
-  if (flags & O_RDWR) {
-    // NOTE - should not normally be checking policy in the middleman, but the handling of Unix flags in DFS is not
-    // consistent right now. 2008-07-16
-    syslog(LOG_ERR, "ERROR: trying to open a file with O_RDWR and DFS does not support that %s dfs %s:%d\n", path,__FILE__, __LINE__);
-    return -EIO;
-  }
-
-  //  fprintf(stderr,"hdfsOpenFile being called %s,%o\n",path,flags);
-
-  // retrieve dfs specific data
-  fi->fh = (uint64_t)hdfsOpenFile(dfs->fs, path, flags,  0, 3, 0);
-
-  if (0 == fi->fh) {
-    syslog(LOG_ERR, "ERROR: could not open file %s dfs %s:%d\n", path,__FILE__, __LINE__);
-    ret = -EIO;
-  }
-
-#endif
 
   return ret;
 }
@@ -935,35 +1060,16 @@
   assert(dfs);
   assert('/' == *path);
 
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-#ifdef OPTIMIZED_READS
-
   dfs_fh *fh = (dfs_fh*)fi->fh;
   hdfsFile file_handle = (hdfsFile)fh->hdfsFH;
 
-#else
-  hdfsFile file_handle = (hdfsFile)fi->fh;
-
-  if (NULL == file_handle) {
-    syslog(LOG_ERR, "ERROR: fuse problem - no file_handle for %s %s:%d\n",path, __FILE__, __LINE__);
-    return -EIO;
-  }
-#endif
-
-  syslog(LOG_DEBUG,"hdfsTell(dfs,%ld)\n",(long)file_handle);
-  tOffset cur_offset = hdfsTell(dfs->fs, file_handle);
-
+  tOffset cur_offset = hdfsTell(fh->fs, file_handle);
   if (cur_offset != offset) {
     syslog(LOG_ERR, "ERROR: user trying to random access write to a file %d!=%d for %s %s:%d\n",(int)cur_offset, (int)offset,path, __FILE__, __LINE__);
     return -EIO;
   }
 
-
-  tSize length = hdfsWrite(dfs->fs, file_handle, buf, size);
+  tSize length = hdfsWrite(fh->fs, file_handle, buf, size);
 
   if(length <= 0) {
     syslog(LOG_ERR, "ERROR: fuse problem - could not write all the bytes for %s %d!=%d%s:%d\n",path,length,(int)size, __FILE__, __LINE__);
@@ -987,35 +1093,27 @@
   assert(path);
   assert(dfs);
   assert('/' == *path);
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
 
   if (NULL == (void*)fi->fh) {
     return  0;
   }
 
-#ifdef OPTIMIZED_READS
   dfs_fh *fh = (dfs_fh*)fi->fh;
   hdfsFile file_handle = (hdfsFile)fh->hdfsFH;
-  free(fh->buf);
-  free(fh);
-#else
-  hdfsFile file_handle = (hdfsFile)fi->fh;
-#endif
 
   if (NULL == file_handle) {
     return 0;
   }
 
-  if (hdfsCloseFile(dfs->fs, file_handle) != 0) {
+  if (hdfsCloseFile(fh->fs, file_handle) != 0) {
     syslog(LOG_ERR, "ERROR: dfs problem - could not close file_handle(%ld) for %s %s:%d\n",(long)file_handle,path, __FILE__, __LINE__);
-    //    fprintf(stderr, "ERROR: dfs problem - could not close file_handle(%ld) for %s %s:%d\n",(long)file_handle,path, __FILE__, __LINE__);
+    fprintf(stderr, "ERROR: dfs problem - could not close file_handle(%ld) for %s %s:%d\n",(long)file_handle,path, __FILE__, __LINE__);
     return -EIO;
   }
 
+  free(fh->buf);
+  free(fh);
+
   fi->fh = (uint64_t)0;
   return 0;
 }
@@ -1041,13 +1139,6 @@
   assert(dfs);
   assert('/' == *path);
 
-
-  // 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))) {
-    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
-    return -EIO;
-  }
-
   if (NULL == (void*)fi->fh) {
     return  0;
   }
@@ -1055,14 +1146,10 @@
   // note that fuse calls flush on RO files too and hdfs does not like that and will return an error
   if(fi->flags & O_WRONLY) {
 
-#ifdef OPTIMIZED_READS
     dfs_fh *fh = (dfs_fh*)fi->fh;
     hdfsFile file_handle = (hdfsFile)fh->hdfsFH;
-#else
-    hdfsFile file_handle = (hdfsFile)fi->fh;
-#endif
 
-    if (hdfsFlush(dfs->fs, file_handle) != 0) {
+    if (hdfsFlush(fh->fs, file_handle) != 0) {
       syslog(LOG_ERR, "ERROR: dfs problem - could not flush file_handle(%lx) for %s %s:%d\n",(long)file_handle,path, __FILE__, __LINE__);
       return -EIO;
     }
@@ -1071,12 +1158,43 @@
   return 0;
 }
 
+static int dfs_access(const char *path, int mask)
+{
+  // 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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+  //  return hdfsAccess(userFS, path, mask);
+  return 0;
+}
 
-void dfs_setattr(struct stat *attr, int to_set, struct fuse_file_info *fi)
+static int dfs_truncate(const char *path, off_t size)
 {
+  (void)path;
+  (void)size;
+  // bugbug we need the FileSystem to support this posix API
+  return -ENOTSUP;
+}
+
 
+static int dfs_symlink(const char *from, const char *to)
+{
+  (void)from;
+  (void)to;
+  // bugbug we need the FileSystem to support this posix API
+  return -ENOTSUP;
 }
 
+
 void dfs_destroy (void *ptr)
 {
   dfs_context *dfs = (dfs_context*)ptr;
@@ -1148,7 +1266,6 @@
 }
 
 
-
 void *dfs_init()
 {
 
@@ -1195,6 +1312,7 @@
   .init         = dfs_init,
   .open	        = dfs_open,
   .read	        = dfs_read,
+  .symlink	= dfs_symlink,
   .statfs	= dfs_statfs,
   .mkdir	= dfs_mkdir,
   .rmdir	= dfs_rmdir,
@@ -1204,11 +1322,11 @@
   .create       = dfs_create,
   .write	= dfs_write,
   .flush        = dfs_flush,
-  //.xsetattr      = dfs_setattr,
   .mknod        = dfs_mknod,
+	.utimens	= dfs_utimens,
   .chmod	= dfs_chmod,
   .chown	= dfs_chown,
-  //  .truncate	= dfs_truncate,
+  .truncate	= dfs_truncate,
 };
 
 
@@ -1254,6 +1372,23 @@
   }
 
 
+  // 
+  // Check we can connect to hdfs
+  // 
+  if (options.initchecks == 1) {
+    hdfsFS temp;
+    if((temp = hdfsConnect(options.server, options.port)) == NULL) {
+      const char *cp = getenv("CLASSPATH");
+      const char *ld = getenv("LD_LIBRARY_PATH");
+      fprintf(stderr, "FATAL: misconfiguration problem, cannot connect to hdfs - here's your environment\n");
+      fprintf(stderr, "LD_LIBRARY_PATH=%s\n",ld == NULL ? "NULL" : ld);
+      fprintf(stderr, "CLASSPATH=%s\n",cp == NULL ? "NULL" : cp);
+      exit(1);
+    }  
+    hdfsDisconnect(temp);
+    temp = NULL;
+  }
+
   int ret = fuse_main(args.argc, args.argv, &dfs_oper, NULL);
 
   if (ret) printf("\n");

Modified: hadoop/core/trunk/src/contrib/fuse-dfs/src/test/TestFuseDFS.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/test/TestFuseDFS.java?rev=696777&r1=696776&r2=696777&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/test/TestFuseDFS.java (original)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/test/TestFuseDFS.java Thu Sep 18 12:34:28 2008
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-import org.apache.hadoop.dfs.*;
 import org.apache.hadoop.hdfs.*;
 import junit.framework.TestCase;
 import java.io.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.*;
 import java.net.*;
 
 /**
@@ -57,7 +57,7 @@
     String lp = System.getProperty("LD_LIBRARY_PATH") + ":" + "/usr/local/lib:" + libhdfs + ":" + jvm;
     System.err.println("LD_LIBRARY_PATH=" + lp);
     String cmd[] =  {  fuse_cmd, "dfs://" + dfs.getHost() + ":" + String.valueOf(dfs.getPort()), 
-		       mountpoint, "-obig_writes", "-odebug", "-oentry_timeout=1",  "-oattribute_timeout=1", "-ousetrash", "rw" };
+                       mountpoint, "-obig_writes", "-odebug", "-oentry_timeout=1",  "-oattribute_timeout=1", "-ousetrash", "rw", "-oinitchecks" };
     final String [] envp = {
       "CLASSPATH="+  cp,
       "LD_LIBRARY_PATH=" + lp,
@@ -255,6 +255,132 @@
       e.printStackTrace();
     }
   }
+  /**
+   * use shell to create a dir and then use filesys to see it exists.
+   */
+  public void testDF() throws IOException,InterruptedException, Exception  {
+    try {
+      // First create a new directory with mkdirs
+      Path path = new Path("/foo");
+      Runtime r = Runtime.getRuntime();
+      String cmd = "df -kh " + mpoint + path.toString();
+      Process p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      InputStream i = p.getInputStream();
+      byte b[] = new byte[i.available()];
+      int length = i.read(b);
+      System.err.println("df output=");
+      System.err.write(b,0,b.length);
+      System.err.println("done");
+
+    } catch(Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  /**
+   * use shell to create a dir and then use filesys to see it exists.
+   */
+  public void testChown() throws IOException,InterruptedException, Exception  {
+    try {
+      // First create a new directory with mkdirs
+      Path path = new Path("/foo");
+      Runtime r = Runtime.getRuntime();
+      String cmd = "mkdir -p " + mpoint + path.toString();
+      Process p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      // check it is there
+      assertTrue(fileSys.getFileStatus(path).isDir());
+
+      cmd = "chown nobody " + mpoint + path.toString();
+      p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      cmd = "chgrp nobody " + mpoint + path.toString();
+      p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      try { Thread.sleep(1000); } catch(Exception e) { }
+
+      FileStatus foo = fileSys.getFileStatus(path);
+
+      assertTrue(foo.getOwner().equals("nobody"));
+      assertTrue(foo.getGroup().equals("nobody"));
+
+    } catch(Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  /**
+   * use shell to create a dir and then use filesys to see it exists.
+   */
+  public void testChmod() throws IOException,InterruptedException, Exception  {
+    try {
+      // First create a new directory with mkdirs
+      Path path = new Path("/foo");
+      Runtime r = Runtime.getRuntime();
+      String cmd = "mkdir -p " + mpoint + path.toString();
+      Process p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      // check it is there
+      assertTrue(fileSys.getFileStatus(path).isDir());
+
+      cmd = "chmod 777 " + mpoint + path.toString();
+      p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      FileStatus foo = fileSys.getFileStatus(path);
+      FsPermission perm = foo.getPermission();
+      assertTrue(perm.toShort() == 0777);
+
+    } catch(Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  /**
+   * use shell to create a dir and then use filesys to see it exists.
+   */
+  public void testUtimes() throws IOException,InterruptedException, Exception  {
+    try {
+      // First create a new directory with mkdirs
+      Path path = new Path("/utimetest");
+      Runtime r = Runtime.getRuntime();
+      String cmd = "touch " + mpoint + path.toString();
+      Process p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      // check it is there
+      assertTrue(fileSys.exists(path));
+
+      FileStatus foo = fileSys.getFileStatus(path);
+      long oldTime = foo.getModificationTime();
+      try { Thread.sleep(1000); } catch(Exception e) {}
+
+      cmd = "touch " + mpoint + path.toString();
+      p = r.exec(cmd);
+      assertTrue(p.waitFor() == 0);
+
+      try { Thread.sleep(1000); } catch(Exception e) {}
+      foo = fileSys.getFileStatus(path);
+      long newTime = foo.getModificationTime();
+
+      assertTrue(newTime > oldTime);
+
+    } catch(Exception e) {
+      e.printStackTrace();
+      throw e;
+    } finally {
+    }
+  }
+
 
 
   /**
@@ -288,11 +414,10 @@
     } catch(Exception e) {
       e.printStackTrace();
     } finally {
-      close();
+    close();
     }
   }
 
-
   /**
    * Unmount and close
    */
@@ -331,7 +456,7 @@
 
       fuse_process.destroy();
       fuse_process = null;
-      if(fileSys != null) {
+        if(fileSys != null) {
         fileSys.close();
         fileSys = null;
       }