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 ma...@apache.org on 2008/12/04 02:45:25 UTC

svn commit: r723185 [2/2] - in /hadoop/core/trunk: ./ src/contrib/fuse-dfs/src/ src/contrib/fuse-dfs/src/test/

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_read.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_read.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_read.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_read.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_file_handle.h"
+
+static size_t min(const size_t x, const size_t y) {
+  return x < y ? x : y;
+}
+
+/**
+ * dfs_read
+ *
+ * Reads from dfs or the open file's buffer.  Note that fuse requires that
+ * either the entire read be satisfied or the EOF is hit or direct_io is enabled
+ *
+ */
+int dfs_read(const char *path, char *buf, size_t size, off_t offset,
+                   struct fuse_file_info *fi)
+{
+  TRACE1("read",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(buf);
+  assert(offset >= 0);
+  assert(size >= 0);
+  assert(fi);
+
+  dfs_fh *fh = (dfs_fh*)fi->fh;
+
+  assert(fh != NULL);
+  assert(fh->fs != NULL);
+  assert(fh->hdfsFH != NULL);
+
+  // special case this as simplifies the rest of the logic to know the caller wanted > 0 bytes
+  if (size == 0)
+    return 0;
+
+  // If size is bigger than the read buffer, then just read right into the user supplied buffer
+  if ( size >= dfs->rdbuffer_size) {
+    int num_read;
+    size_t total_read = 0;
+    while (size - total_read > 0 && (num_read = hdfsPread(fh->fs, fh->hdfsFH, offset + total_read, buf + total_read, size - total_read)) > 0) {
+      total_read += num_read;
+    }
+    // if there was an error before satisfying the current read, this logic declares it an error
+    // and does not try to return any of the bytes read. Don't think it matters, so the code
+    // is just being conservative.
+    if (total_read < size && num_read < 0) {
+      total_read = -EIO;
+    }
+    return total_read;
+  }
+
+  //
+  // Critical section - protect from multiple reads in different threads accessing the read buffer
+  // (no returns until end)
+  //
+
+  pthread_mutex_lock(&fh->mutex);
+
+  // used only to check the postcondition of this function - namely that we satisfy
+  // the entire read or EOF is hit.
+  int isEOF = 0;
+  int ret = 0;
+
+  // check if the buffer is empty or
+  // the read starts before the buffer starts or
+  // the read ends after the buffer ends
+
+  if (fh->bufferSize == 0  || 
+      offset < fh->buffersStartOffset || 
+      offset + size > fh->buffersStartOffset + fh->bufferSize) 
+    {
+      // Read into the buffer from DFS
+      int num_read = 0;
+      size_t total_read = 0;
+
+      while (dfs->rdbuffer_size  - total_read > 0 &&
+             (num_read = hdfsPread(fh->fs, fh->hdfsFH, offset + total_read, fh->buf + total_read, dfs->rdbuffer_size - total_read)) > 0) {
+        total_read += num_read;
+      }
+
+      // if there was an error before satisfying the current read, this logic declares it an error
+      // and does not try to return any of the bytes read. Don't think it matters, so the code
+      // is just being conservative.
+      if (total_read < size && num_read < 0) {
+        // invalidate the buffer 
+        fh->bufferSize = 0; 
+        syslog(LOG_ERR, "Read error - pread failed for %s with return code %d %s:%d", path, (int)num_read, __FILE__, __LINE__);
+        ret = -EIO;
+      } else {
+        // Either EOF, all read or read beyond size, but then there was an error
+        fh->bufferSize = total_read;
+        fh->buffersStartOffset = offset;
+
+        if (dfs->rdbuffer_size - total_read > 0) {
+          // assert(num_read == 0); this should be true since if num_read < 0 handled above.
+          isEOF = 1;
+        }
+      }
+    }
+
+  //
+  // NOTE on EOF, fh->bufferSize == 0 and ret = 0 ,so the logic for copying data into the caller's buffer is bypassed, and
+  //  the code returns 0 as required
+  //
+  if (ret == 0 && fh->bufferSize > 0) {
+
+    assert(offset >= fh->buffersStartOffset);
+    assert(fh->buf);
+
+    const size_t bufferReadIndex = offset - fh->buffersStartOffset;
+    assert(bufferReadIndex >= 0 && bufferReadIndex < fh->bufferSize);
+
+    const size_t amount = min(fh->buffersStartOffset + fh->bufferSize - offset, size);
+    assert(amount >= 0 && amount <= fh->bufferSize);
+
+    const char *offsetPtr = fh->buf + bufferReadIndex;
+    assert(offsetPtr >= fh->buf);
+    assert(offsetPtr + amount <= fh->buf + fh->bufferSize);
+    
+    memcpy(buf, offsetPtr, amount);
+
+    ret = amount;
+  }
+
+  //
+  // Critical section end 
+  //
+  pthread_mutex_unlock(&fh->mutex);
+ 
+  // fuse requires the below and the code should guarantee this assertion
+  // 3 cases on return:
+  //   1. entire read satisfied
+  //   2. partial read and isEOF - including 0 size read
+  //   3. error 
+  assert(ret == size || isEOF || ret < 0);
+
+ return ret;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_readdir.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_readdir.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_readdir.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_readdir.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_stat_struct.h"
+#include "fuse_connect.h"
+
+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
+  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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  // call dfs to read the dir
+  int numEntries = 0;
+  hdfsFileInfo *info = hdfsListDirectory(userFS,path,&numEntries);
+  userFS = NULL;
+
+  // NULL means either the directory doesn't exist or maybe IO error.
+  if (NULL == info) {
+    return -ENOENT;
+  }
+
+  int i ;
+  for (i = 0; i < numEntries; i++) {
+
+    // check the info[i] struct
+    if (NULL == info[i].mName) {
+      syslog(LOG_ERR,"ERROR: for <%s> info[%d].mName==NULL %s:%d", path, i, __FILE__,__LINE__);
+      continue;
+    }
+
+    struct stat st;
+    fill_stat_structure(&info[i], &st);
+
+    // hack city: todo fix the below to something nicer and more maintainable but
+    // with good performance
+    // strip off the path but be careful if the path is solely '/'
+    // NOTE - this API started returning filenames as full dfs uris
+    const char *const str = info[i].mName + dfs->dfs_uri_len + path_len + ((path_len == 1 && *path == '/') ? 0 : 1);
+
+    // pack this entry into the fuse buffer
+    int res = 0;
+    if ((res = filler(buf,str,&st,0)) != 0) {
+      syslog(LOG_ERR, "ERROR: readdir filling the buffer %d %s:%d\n",res, __FILE__, __LINE__);
+    }
+  }
+
+  // insert '.' and '..'
+  const char *const dots [] = { ".",".."};
+  for (i = 0 ; i < 2 ; i++)
+    {
+      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 =  0;
+
+      // setup stat size and acl meta data
+      st.st_size    = 512;
+      st.st_blksize = 512;
+      st.st_blocks  =  1;
+      st.st_mode    = (S_IFDIR | 0777);
+      st.st_uid     = default_id;
+      st.st_gid     = default_id;
+      // todo fix below times
+      st.st_atime   = 0;
+      st.st_mtime   = 0;
+      st.st_ctime   = 0;
+
+      const char *const str = dots[i];
+
+      // flatten the info using fuse's function into a buffer
+      int res = 0;
+      if ((res = filler(buf,str,&st,0)) != 0) {
+        syslog(LOG_ERR, "ERROR: readdir filling the buffer %d %s:%d", res, __FILE__, __LINE__);
+      }
+    }
+  // free the info pointers
+  hdfsFreeFileInfo(info,numEntries);
+  return 0;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_release.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_release.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_release.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_release.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_file_handle.h"
+
+/**
+ * This mutex is to protect releasing a file handle in case the user calls close in different threads
+ * and fuse passes these calls to here.
+ */
+pthread_mutex_t release_mutex = PTHREAD_MUTEX_INITIALIZER;
+
+int dfs_release (const char *path, struct fuse_file_info *fi) {
+  TRACE1("release", 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);
+
+  int ret = 0;
+
+  //
+  // Critical section - protect from multiple close calls in different threads.
+  // (no returns until end)
+  //
+
+  pthread_mutex_lock(&release_mutex);
+
+  if (NULL != (void*)fi->fh) {
+
+    dfs_fh *fh = (dfs_fh*)fi->fh;
+    assert(fh);
+
+    hdfsFile file_handle = (hdfsFile)fh->hdfsFH;
+
+    if (NULL != file_handle) {
+      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__);
+        ret = -EIO;
+      }
+    }
+
+    if (fh->buf != NULL) {
+      free(fh->buf);
+    }
+    // this is always created and initialized, so always destroy it. (see dfs_open)
+    pthread_mutex_destroy(&fh->mutex);
+
+    free(fh);
+
+    fi->fh = (uint64_t)0;
+  }
+
+  pthread_mutex_unlock(&release_mutex);
+
+  //
+  // End critical section 
+  // 
+
+  return ret;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rename.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rename.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rename.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rename.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_trash.h"
+#include "fuse_connect.h"
+
+int dfs_rename(const char *from, const char *to)
+{
+  TRACE1("rename", from) 
+
+ // retrieve dfs specific data
+  dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
+
+  // check params and the context var
+  assert(from);
+  assert(to);
+  assert(dfs);
+
+  assert('/' == *from);
+  assert('/' == *to);
+
+  if (is_protected(from) || is_protected(to)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to rename: %s %s", from, to);
+    return -EACCES;
+  }
+
+  if (dfs->read_only) {
+    syslog(LOG_ERR,"ERROR: hdfs is configured as read-only, cannot rename the directory %s\n",from);
+    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 (hdfsRename(userFS, from, to)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to rename %s to %s",from, to);
+    return -EIO;
+  }
+
+  return 0;
+
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_trash.h"
+#include "fuse_connect.h"
+
+extern const char *const TrashPrefixDir;
+
+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);
+
+  if (is_protected(path)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to delete a protected directory: %s ",path);
+    return -EACCES;
+  }
+
+  if (dfs->read_only) {
+    syslog(LOG_ERR,"ERROR: hdfs is configured as read-only, cannot delete 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;
+  }
+
+  int numEntries = 0;
+  hdfsFileInfo *info = hdfsListDirectory(userFS,path,&numEntries);
+
+  // free the info pointers
+  hdfsFreeFileInfo(info,numEntries);
+
+  if (numEntries) {
+    return -ENOTEMPTY;
+  }
+
+  if (hdfsDeleteWithTrash(userFS, path, dfs->usetrash)) {
+    syslog(LOG_ERR,"ERROR: hdfs error trying to delete the directory %s\n",path);
+    return -EIO;
+  }
+
+  return 0;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_statfs.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_statfs.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_statfs.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_statfs.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_connect.h"
+
+
+int dfs_statfs(const char *path, struct statvfs *st)
+{
+  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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  const long cap   = hdfsGetCapacity(userFS);
+  const long used  = hdfsGetUsed(userFS);
+  const long 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
+     };
+  */
+
+  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;
+  st->f_fsid    =  1023;
+  st->f_flag    =  ST_RDONLY | ST_NOSUID;
+  st->f_namemax =  1023;
+
+  return 0;
+}
+

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_symlink.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_symlink.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_symlink.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_symlink.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+
+
+int dfs_symlink(const char *from, const char *to)
+{
+  TRACE1("symlink", from)
+  (void)from;
+  (void)to;
+  // bugbug we need the FileSystem to support this posix API
+  return -ENOTSUP;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_truncate.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_truncate.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_truncate.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_truncate.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_connect.h"
+
+/**
+ * For now implement truncate here and only for size == 0.
+ * Weak implementation in that we just delete the file and 
+ * then re-create it, but don't set the user, group, and times to the old
+ * file's metadata. 
+ */
+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;
+
+  assert(path);
+  assert('/' == *path);
+  assert(dfs);
+
+  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) {
+    syslog(LOG_ERR, "ERROR: could not connect to dfs %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  int flags = O_WRONLY | O_CREAT;
+
+  hdfsFile file;
+  if ((file = (hdfsFile)hdfsOpenFile(userFS, path, flags,  0, 3, 0)) == NULL) {
+    syslog(LOG_ERR, "ERROR: could not connect open file %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  if (hdfsCloseFile(userFS, file) != 0) {
+    syslog(LOG_ERR, "ERROR: could not connect close file %s:%d\n", __FILE__, __LINE__);
+    return -EIO;
+  }
+  return 0;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_unlink.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_unlink.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_unlink.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_unlink.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_connect.h"
+#include "fuse_trash.h"
+extern const char *const TrashPrefixDir;
+
+int dfs_unlink(const char *path)
+{
+  TRACE1("unlink", 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);
+
+  if (is_protected(path)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to delete a protected directory: %s ",path);
+    return -EACCES;
+  }
+
+  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 (hdfsDeleteWithTrash(userFS, path, dfs->usetrash)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to delete the file %s",path);
+    return -EIO;
+  }
+
+  return 0;
+
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_utimens.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_utimens.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_utimens.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_utimens.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_connect.h"
+
+ int dfs_utimens(const char *path, const struct timespec ts[2])
+{
+  TRACE1("utimens", path)
+#if PERMS
+  // 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;
+  }
+#endif  
+  return 0;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_write.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_write.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_write.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_impls_write.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_impls.h"
+#include "fuse_file_handle.h"
+
+int dfs_write(const char *path, const char *buf, size_t size,
+                     off_t offset, struct fuse_file_info *fi)
+{
+  TRACE1("write", path)
+
+  // retrieve dfs specific data
+  dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
+  int ret = 0;
+
+  // check params and the context var
+  assert(path);
+  assert(dfs);
+  assert('/' == *path);
+  assert(fi);
+
+  dfs_fh *fh = (dfs_fh*)fi->fh;
+  assert(fh);
+
+  hdfsFile file_handle = (hdfsFile)fh->hdfsFH;
+  assert(file_handle);
+
+  //
+  // Critical section - make the sanity check (tell to see the writes are sequential) and the actual write 
+  // (no returns until end)
+  //
+  pthread_mutex_lock(&fh->mutex);
+
+  tSize length = 0;
+  assert(fh->fs);
+
+  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__);
+    ret =  -EIO;
+  } else {
+    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__);
+      ret = -EIO;
+    } 
+    if (length != size) {
+      syslog(LOG_ERR, "WARN: fuse problem - could not write all the bytes for %s %d!=%d%s:%d\n",path,length,(int)size, __FILE__, __LINE__);
+    }
+  }
+
+  //
+  // Critical section end 
+  //
+
+  pthread_mutex_unlock(&fh->mutex);
+
+  return ret == 0 ? length : ret;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <strings.h>
+
+#include "fuse_dfs.h"
+#include "fuse_init.h"
+#include "fuse_options.h"
+#include "fuse_context_handle.h"
+
+// Hacked up function to basically do:
+//  protectedpaths = split(options.protected,':');
+
+void init_protectedpaths(dfs_context *dfs) {
+
+  char *tmp = options.protected;
+
+
+  // handle degenerate case up front.
+  if (tmp == NULL || 0 == *tmp) {
+    dfs->protectedpaths = (char**)malloc(sizeof(char*));
+    dfs->protectedpaths[0] = NULL;
+    return;
+  }
+  assert(tmp);
+
+  if (options.debug) {
+    print_options();
+  }
+
+
+  int i = 0;
+  while (tmp && (NULL != (tmp = index(tmp,':')))) {
+    tmp++; // pass the ,
+    i++;
+  }
+  i++; // for the last entry
+  i++; // for the final NULL
+  dfs->protectedpaths = (char**)malloc(sizeof(char*)*i);
+  assert(dfs->protectedpaths);
+  tmp = options.protected;
+  int j  = 0;
+  while (NULL != tmp && j < i) {
+    int length;
+    char *eos = index(tmp,':');
+    if (NULL != eos) {
+      length = eos - tmp; // length of this value
+    } else {
+      length = strlen(tmp);
+    }
+    dfs->protectedpaths[j] = (char*)malloc(sizeof(char)*length+1);
+    assert(dfs->protectedpaths[j]);
+    strncpy(dfs->protectedpaths[j], tmp, length);
+    dfs->protectedpaths[j][length] = '\0';
+    if (eos) {
+      tmp = eos + 1;
+    } else {
+      tmp = NULL;
+    }
+    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()
+{
+
+  //
+  // 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));
+
+  if (NULL == dfs) {
+    syslog(LOG_ERR, "FATAL: could not malloc fuse dfs context struct - out of memory %s:%d", __FILE__, __LINE__);
+    exit(1);
+  }
+
+  // initialize the context
+  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;
+  dfs->rdbuffer_size         = options.rdbuffer_size;
+  dfs->direct_io             = options.direct_io;
+
+  bzero(dfs->dfs_uri,0);
+  sprintf(dfs->dfs_uri,"dfs://%s:%d/",dfs->nn_hostname,dfs->nn_port);
+  dfs->dfs_uri_len = strlen(dfs->dfs_uri);
+
+  // use ERR level to ensure it makes it into the log.
+  syslog(LOG_ERR, "mounting %s", dfs->dfs_uri);
+
+  init_protectedpaths(dfs);
+  assert(dfs->protectedpaths != NULL);
+
+  if (dfs->rdbuffer_size <= 0) {
+    syslog(LOG_DEBUG, "WARN: dfs->rdbuffersize <= 0 = %ld %s:%d", dfs->rdbuffer_size, __FILE__, __LINE__);
+    dfs->rdbuffer_size = 32768;
+  }
+  return (void*)dfs;
+}
+
+
+
+void dfs_destroy (void *ptr)
+{
+  TRACE("destroy")
+  dfs_context *dfs = (dfs_context*)ptr;
+  dfs->fs = NULL;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.h
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.h?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.h (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_init.h Wed Dec  3 17:45:23 2008
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __FUSE_INIT_H__
+#define __FUSE_INIT_H__
+
+/**
+ * These are responsible for initializing connections to dfs and internal
+ * data structures and then freeing them.
+ * i.e., what happens on mount and unmount.
+ *
+ */
+void *dfs_init();
+void dfs_destroy (void *ptr);
+
+#endif

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "fuse_dfs.h"
+#include "fuse_options.h"
+#include <getopt.h>
+
+#include "fuse_context_handle.h"
+
+void print_options() {
+  fprintf(stderr,"options:\n");
+  fprintf(stderr, "\tprotected=%s\n",options.protected);
+  fprintf(stderr, "\tserver=%s\n",options.server);
+  fprintf(stderr, "\tport=%d\n",options.port);
+  fprintf(stderr, "\tdebug=%d\n",options.debug);
+  fprintf(stderr, "\tread_only=%d\n",options.read_only);
+  fprintf(stderr, "\tusetrash=%d\n",options.usetrash);
+  fprintf(stderr, "\tentry_timeout=%d\n",options.entry_timeout);
+  fprintf(stderr, "\tattribute_timeout=%d\n",options.attribute_timeout);
+  fprintf(stderr, "\tprivate=%d\n",options.private);
+  fprintf(stderr, "\trdbuffer_size=%d (KBs)\n",(int)options.rdbuffer_size/1024);
+}
+
+const char *program;  
+
+
+/** macro to define options */
+#define DFSFS_OPT_KEY(t, p, v) { t, offsetof(struct options, p), v }
+
+void print_usage(const char *pname)
+{
+  fprintf(stdout,"USAGE: %s [debug] [--help] [--version] [-oprotected=<colon_seped_list_of_paths] [rw] [-onotrash] [-ousetrash] [-obig_writes] [-oprivate (single user)] [ro] [-oserver=<hadoop_servername>] [-oport=<hadoop_port>] [-oentry_timeout=<secs>] [-oattribute_timeout=<secs>] [-odirect_io] [-onopoermissions] [-o<other fuse option>] <mntpoint> [fuse options]\n",pname);
+  fprintf(stdout,"NOTE: debugging option for fuse is -debug\n");
+}
+
+
+/** keys for FUSE_OPT_ options */
+enum
+  {
+    KEY_VERSION,
+    KEY_HELP,
+    KEY_USETRASH,
+    KEY_NOTRASH,
+    KEY_RO,
+    KEY_RW,
+    KEY_PRIVATE,
+    KEY_BIGWRITES,
+    KEY_DEBUG,
+    KEY_INITCHECKS,
+    KEY_NOPERMISSIONS,
+    KEY_DIRECTIO,
+  };
+
+struct fuse_opt dfs_opts[] =
+  {
+    DFSFS_OPT_KEY("server=%s", server, 0),
+    DFSFS_OPT_KEY("entry_timeout=%d", entry_timeout, 0),
+    DFSFS_OPT_KEY("attribute_timeout=%d", attribute_timeout, 0),
+    DFSFS_OPT_KEY("protected=%s", protected, 0),
+    DFSFS_OPT_KEY("port=%d", port, 0),
+    DFSFS_OPT_KEY("rdbuffer=%d", rdbuffer_size,0),
+
+    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("nopermissions", KEY_NOPERMISSIONS),
+    FUSE_OPT_KEY("big_writes", KEY_BIGWRITES),
+    FUSE_OPT_KEY("rw", KEY_RW),
+    FUSE_OPT_KEY("usetrash", KEY_USETRASH),
+    FUSE_OPT_KEY("notrash", KEY_NOTRASH),
+    FUSE_OPT_KEY("direct_io", KEY_DIRECTIO),
+    FUSE_OPT_KEY("-v",             KEY_VERSION),
+    FUSE_OPT_KEY("--version",      KEY_VERSION),
+    FUSE_OPT_KEY("-h",             KEY_HELP),
+    FUSE_OPT_KEY("--help",         KEY_HELP),
+    FUSE_OPT_END
+  };
+
+int dfs_options(void *data, const char *arg, int key,  struct fuse_args *outargs)
+{
+  (void) data;
+
+  switch (key) {
+  case FUSE_OPT_KEY_OPT:
+    fprintf(stderr,"fuse-dfs ignoring option %s\n",arg);
+    return 1;
+  case  KEY_VERSION:
+    fprintf(stdout,"%s %s\n",program,_FUSE_DFS_VERSION);
+    exit(0);
+  case KEY_HELP:
+    print_usage(program);
+    exit(0);
+  case KEY_USETRASH:
+    options.usetrash = 1;
+    break;
+  case KEY_NOTRASH:
+    options.usetrash = 1;
+    break;
+  case KEY_RO:
+    options.read_only = 1;
+    break;
+  case KEY_RW:
+    options.read_only = 0;
+    break;
+  case KEY_PRIVATE:
+    options.private = 1;
+    break;
+  case KEY_DEBUG:
+    fuse_opt_add_arg(outargs, "-d");
+    options.debug = 1;
+    break;
+  case KEY_INITCHECKS:
+    options.initchecks = 1;
+    break;
+  case KEY_NOPERMISSIONS:
+    options.no_permissions = 1;
+    break;
+  case KEY_DIRECTIO:
+    options.direct_io = 1;
+    break;
+  case KEY_BIGWRITES:
+#ifdef FUSE_CAP_BIG_WRITES
+    fuse_opt_add_arg(outargs, "-obig_writes");
+#endif
+    break;
+  default: {
+    // try and see if the arg is a URI for DFS
+    int tmp_port;
+    char tmp_server[1024];
+
+    if (!sscanf(arg,"dfs://%1024[a-zA-Z0-9_.-]:%d",tmp_server,&tmp_port)) {
+      if (strcmp(arg,"ro") == 0) {
+        options.read_only = 1;
+      } else if (strcmp(arg,"rw") == 0) {
+        options.read_only = 0;
+      } else {
+        fprintf(stderr,"fuse-dfs didn't recognize %s,%d\n",arg,key);
+        fuse_opt_add_arg(outargs,arg);
+        return 0;
+      }
+    } else {
+      options.port = tmp_port;
+      options.server = strdup(tmp_server);
+      fprintf(stderr, "port=%d,server=%s\n", options.port, options.server);
+    }
+  }
+  }
+  return 0;
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.h
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.h?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.h (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_options.h Wed Dec  3 17:45:23 2008
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __FUSE_OPTIONS_H__
+#define __FUSE_OPTIONS_H__
+
+/** options for fuse_opt.h */
+struct options {
+  char* protected;
+  char* server;
+  int port;
+  int debug;
+  int read_only;
+  int initchecks;
+  int no_permissions;
+  int usetrash;
+  int entry_timeout;
+  int attribute_timeout;
+  int private;
+  size_t rdbuffer_size;
+  int direct_io;
+} options;
+
+extern struct fuse_opt dfs_opts[];
+void print_options();
+void print_usage(const char *pname);
+int dfs_options(void *data, const char *arg, int key,  struct fuse_args *outargs);
+
+#endif

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <math.h>
+#include <pthread.h>
+#include <grp.h>
+#include <pwd.h>
+
+#include "fuse_dfs.h"
+#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 blksize = 512;
+
+/**
+ * Converts from a hdfs hdfsFileInfo to a POSIX stat struct
+ *
+ */
+int fill_stat_structure(hdfsFileInfo *info, struct stat *st) 
+{
+  assert(st);
+  assert(info);
+
+  // 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 PERMS
+  if (info->mOwner != NULL) {
+    //
+    // Critical section - protect from concurrent calls in different threads since
+    // the struct below is static.
+    // (no returns until end)
+    //
+    pthread_mutex_lock(&passwdstruct_mutex);
+
+    struct passwd *passwd_info = getpwnam(info->mOwner);
+    owner_id = passwd_info == NULL ? default_id : passwd_info->pw_uid;
+
+    //
+    // End critical section 
+    // 
+    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
+    // the struct below is static.
+    // (no returns until end)
+    //
+    pthread_mutex_lock(&groupstruct_mutex);
+
+    struct group *grp = getgrnam(info->mGroup);
+    group_id = grp == NULL ? default_id : grp->gr_gid;
+
+    //
+    // End critical section 
+    // 
+    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;
+  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;
+#if PERMS
+  st->st_atime    = info->mLastAccess;
+#else
+  st->st_atime    = info->mLastMod;
+#endif
+  st->st_mtime    = info->mLastMod;
+  st->st_ctime    = info->mLastMod;
+
+  return 0;
+}
+

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.h
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.h?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.h (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_stat_struct.h Wed Dec  3 17:45:23 2008
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __FUSE_STAT_STRUCT_H__
+#define __FUSE_STAT_STRUCT_H__
+
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+
+#include "hdfs.h"
+
+/**
+ * Converts from a hdfs hdfsFileInfo to a POSIX stat struct
+ * Should be thread safe.
+ */
+int fill_stat_structure(hdfsFileInfo *info, struct stat *st) ;
+
+extern const int default_id;
+extern const int blksize;
+#endif

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include <hdfs.h>
+#include <strings.h>
+
+#include "fuse_dfs.h"
+#include "fuse_trash.h"
+#include "fuse_context_handle.h"
+
+
+const char *const TrashPrefixDir = "/user/root/.Trash";
+const char *const TrashDir = "/user/root/.Trash/Current";
+
+#define TRASH_RENAME_TRIES  100
+
+//
+// NOTE: this function is a c implementation of org.apache.hadoop.fs.Trash.moveToTrash(Path path).
+//
+
+int move_to_trash(const char *item, hdfsFS userFS) {
+
+  // retrieve dfs specific data
+  dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
+
+  // check params and the context var
+  assert(item);
+  assert(dfs);
+  assert('/' == *item);
+  assert(rindex(item,'/') >= 0);
+
+
+  char fname[4096]; // or last element of the directory path
+  char parent_directory[4096]; // the directory the fname resides in
+
+  if (strlen(item) > sizeof(fname) - strlen(TrashDir)) {
+    syslog(LOG_ERR, "ERROR: internal buffer too small to accomodate path of length %d %s:%d\n", (int)strlen(item), __FILE__, __LINE__);
+    return -EIO;
+  }
+
+  // separate the file name and the parent directory of the item to be deleted
+  {
+    int length_of_parent_dir = rindex(item, '/') - item ;
+    int length_of_fname = strlen(item) - length_of_parent_dir - 1; // the '/'
+
+    // note - the below strncpys should be safe from overflow because of the check on item's string length above.
+    strncpy(parent_directory, item, length_of_parent_dir);
+    parent_directory[length_of_parent_dir ] = 0;
+    strncpy(fname, item + length_of_parent_dir + 1, strlen(item));
+    fname[length_of_fname + 1] = 0;
+  }
+
+  // create the target trash directory
+  char trash_dir[4096];
+  if (snprintf(trash_dir, sizeof(trash_dir), "%s%s",TrashDir,parent_directory) >= sizeof trash_dir) {
+    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;
+  }
+
+  // create the target trash directory in trash (if needed)
+  if ( hdfsExists(userFS, trash_dir)) {
+    // make the directory to put it in in the Trash - NOTE
+    // hdfsCreateDirectory also creates parents, so Current will be created if it does not exist.
+    if (hdfsCreateDirectory(userFS, trash_dir)) {
+      return -EIO;
+    }
+  }
+
+  //
+  // if the target path in Trash already exists, then append with
+  // a number. Start from 1.
+  //
+  char target[4096];
+  int j ;
+  if ( snprintf(target, sizeof target,"%s/%s",trash_dir, fname) >= 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;
+  }
+
+  // NOTE: this loop differs from the java version by capping the #of tries
+  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;
+    }
+  }
+  if (hdfsRename(userFS, item, target)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to rename %s to %s",item, target);
+    return -EIO;
+  }
+  return 0;
+} 
+
+
+int hdfsDeleteWithTrash(hdfsFS userFS, const char *path, int useTrash) {
+
+  // move the file to the trash if this is enabled and its not actually in the trash.
+  if (useTrash && strncmp(path, TrashPrefixDir, strlen(TrashPrefixDir)) != 0) {
+    int ret= move_to_trash(path, userFS);
+    return ret;
+  }
+
+  if (hdfsDelete(userFS, path)) {
+    syslog(LOG_ERR,"ERROR: hdfs trying to delete the file %s",path);
+    return -EIO;
+  }
+  return 0;
+
+}

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.h
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.h?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.h (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_trash.h Wed Dec  3 17:45:23 2008
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __FUSE_TRASH_H__
+#define __FUSE_TRASH_H__
+
+#include <hdfs.h>
+
+int hdfsDeleteWithTrash(hdfsFS userFS, const char *path, int useTrash);
+
+#endif

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.c
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.c?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.c (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.c Wed Dec  3 17:45:23 2008
@@ -0,0 +1,221 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include <pthread.h>
+#include <grp.h>
+#include <pwd.h>
+#include <stdlib.h>
+
+#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)
+{
+  //
+  // Critical section - protect from concurrent calls in different threads.
+  // since the struct below is static.
+  // (no returns until end)
+  //
+
+  pthread_mutex_lock(&passwdstruct_mutex);
+
+  struct passwd *userinfo = getpwuid(uid);
+  char * ret = userinfo && userinfo->pw_name ? strdup(userinfo->pw_name) : NULL;
+
+  pthread_mutex_unlock(&passwdstruct_mutex);
+
+  //
+  // End critical section 
+  // 
+  return ret;
+}
+
+/**
+ * Cleans up a char ** group pointer
+ */
+
+void freeGroups(char **groups, int numgroups) {
+  if (groups == NULL) {
+    return;
+  }
+  int i ;
+  for (i = 0; i < numgroups; i++) {
+    free(groups[i]);
+  }
+  free(groups);
+}
+
+#define GROUPBUF_SIZE 5
+
+char *getGroup(gid_t gid) {
+  //
+  // Critical section - protect from concurrent calls in different threads.
+  // since the struct below is static.
+  // (no returns until end)
+  //
+
+  pthread_mutex_lock(&groupstruct_mutex);
+
+  struct group* grp = getgrgid(gid);
+  char * ret = grp && grp->gr_name ? strdup(grp->gr_name) : NULL;
+
+  //
+  // End critical section 
+  // 
+  pthread_mutex_unlock(&groupstruct_mutex);
+
+  return ret;
+}
+
+
+/**
+ * Utility for getting the group from the uid
+ * NOTE: if non-null return, the return must be freed by the caller.
+ */
+char *getGroupUid(uid_t uid) {
+  //
+  // Critical section - protect from concurrent calls in different threads
+  // since the structs below are static.
+  // (no returns until end)
+  //
+
+  pthread_mutex_lock(&passwdstruct_mutex);
+  pthread_mutex_lock(&groupstruct_mutex);
+
+  char *ret = NULL;
+  struct passwd *userinfo = getpwuid(uid);
+  if (NULL != userinfo) {
+    struct group* grp = getgrgid( userinfo->pw_gid);
+    ret = grp && grp->gr_name ? strdup(grp->gr_name) : NULL;
+  }
+
+  //
+  // End critical section 
+  // 
+  pthread_mutex_unlock(&groupstruct_mutex);
+  pthread_mutex_unlock(&passwdstruct_mutex);
+
+  return ret;
+}
+
+
+/**
+ * lookup the gid based on the uid
+ */
+gid_t getGidUid(uid_t uid) {
+  //
+  // Critical section - protect from concurrent calls in different threads
+  // since the struct below is static.
+  // (no returns until end)
+  //
+
+  pthread_mutex_lock(&passwdstruct_mutex);
+
+  struct passwd *userinfo = getpwuid(uid);
+  gid_t gid = userinfo == NULL ? 0 : userinfo->pw_gid;
+
+  //
+  // End critical section 
+  // 
+  pthread_mutex_unlock(&passwdstruct_mutex);
+
+  return gid;
+}
+
+/**
+ * Utility for getting the groups for the user making the fuse call in char * form
+ */
+char ** getGroups(uid_t uid, int *num_groups)
+{
+  char *user = getUsername(uid);
+
+  if (user == NULL)
+    return NULL;
+
+  char **groupnames = NULL;
+
+  // see http://www.openldap.org/lists/openldap-devel/199903/msg00023.html
+
+  //#define GETGROUPS_T 1 
+#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;
+
+  gid_t gid = getGidUid(uid);
+
+  if ((rtr = getgrouplist(user, 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(user, gid, grouplist, num_groups);
+    }
+  }
+
+  groupnames = (char**)malloc(sizeof(char*)* (*num_groups) + 1);
+  assert(groupnames);
+  int i;
+  for (i=0; i < *num_groups; i++)  {
+    groupnames[i] = getGroup(grouplist[i]);
+    if (groupnames[i] == NULL) {
+      fprintf(stderr, "error could not lookup group %d\n",(int)grouplist[i]);
+    }
+  } 
+  free(grouplist);
+  assert(user != NULL);
+  groupnames[i] = user;
+  *num_groups = *num_groups + 1;
+#else
+
+  int i = 0;
+  assert(user != NULL);
+  groupnames[i] = user;
+  i++;
+
+  groupnames[i] = getGroupUid(uid);
+  if (groupnames[i]) {
+    i++;
+  }
+
+  *num_groups = i;
+
+#endif
+  return groupnames;
+}
+#endif

Added: hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.h
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.h?rev=723185&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.h (added)
+++ hadoop/core/trunk/src/contrib/fuse-dfs/src/fuse_users.h Wed Dec  3 17:45:23 2008
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __FUSE_USERS_H__
+#define __FUSE_USERS_H__
+
+#include <grp.h>
+#include <pwd.h>
+#include <pthread.h>
+
+/**
+ * Overall Note:
+ * 1. all these functions should be thread safe.
+ * 2. the ones that return char * or char **, generally require
+ * the caller to free the return value.
+ *
+ */
+
+
+/**
+ * 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);
+
+
+/**
+ * Cleans up a char ** group pointer
+ */
+void freeGroups(char **groups, int numgroups);
+
+/**
+ * Lookup single group. Caller responsible for free of the return value
+ */
+char *getGroup(gid_t gid);
+
+/**
+ * Utility for getting the group from the uid
+ * NOTE: if non-null return, the return must be freed by the caller.
+ */
+char *getGroupUid(uid_t uid) ;
+
+
+/**
+ * lookup the gid based on the uid
+ */
+
+gid_t getGidUid(uid_t uid);
+
+/**
+ * Utility for getting the groups for the user making the fuse call in char * form
+ */
+char ** getGroups(uid_t uid, int *num_groups);
+
+#endif

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=723185&r1=723184&r2=723185&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 Wed Dec  3 17:45:23 2008
@@ -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", "-oinitchecks",
+                       mountpoint, "-obig_writes", "-odebug", "-oentry_timeout=0.1",  "-oattribute_timeout=0.1", "-ousetrash", "rw", "-oinitchecks",
                        "-ordbuffer=32768"};
     final String [] envp = {
       "CLASSPATH="+  cp,
@@ -169,6 +169,12 @@
       f.flush();
       f.close();
 
+
+      try {
+        Thread.sleep(1000);
+      } catch(Exception e) {
+      }
+
       // check the file exists.
       Path myPath = new Path("/hello.txt");
       assertTrue(fileSys.exists(myPath));
@@ -177,15 +183,15 @@
       FileInputStream fi = new FileInputStream(new File(mpoint, "hello.txt"));
       byte b[] = new byte[12];
       int length = fi.read(b,0,12);
-      String s2 = new String( b);
+      assertTrue(length > 0);
+      String s2 = new String( b, 0, length);
+      assertEquals("hello world", s2);
     } catch(Exception e) {
       e.printStackTrace();
     } finally {
     }
   }
 
-
-
   /**
    * Test ls for dir already created in testMkdDir also tests bad ls
    */
@@ -256,6 +262,7 @@
       e.printStackTrace();
     }
   }
+
   /**
    * use shell to create a dir and then use filesys to see it exists.
    */
@@ -505,8 +512,65 @@
     }
   }
 
+
+
+  /**
+   * Use filesys to create the hello world! file and then cat it and see its contents are correct.
+   */
+  public void testAppends() throws IOException,InterruptedException  {
+    try {
+      // First create a new directory with mkdirs
+      Runtime r = Runtime.getRuntime();
+
+      {
+        FileOutputStream os = new FileOutputStream(mpoint + "/appends");
+        String hello = "hello";
+        os.write(hello.getBytes());
+        os.flush();
+        os.close();
+      }
+
+      // check it exists
+      Path myPath = new Path("/appends");
+      assertTrue(fileSys.exists(myPath));
+
+      try {
+        Thread.sleep(1000);
+      } catch(Exception e) {
+      }
+
+      FileStatus foo = fileSys.getFileStatus(myPath);
+
+      File f = new File(mpoint + "/appends");
+      assertTrue(f.length() > 0);
+
+      {
+        FileOutputStream os = new FileOutputStream(mpoint + "/appends", true);
+        String hello = " world!";
+        os.write(hello.getBytes());
+        os.flush();
+        os.close();
+      }
+
+      // cat the file
+      FileInputStream is = new FileInputStream(mpoint + "/appends");
+      byte b[] = new byte[1024];
+      int len = is.read(b);
+      assertTrue(len > 0);
+      String s2 = new String(b,0,len);
+      assertTrue(s2.equals("hello world!"));
+
+    } catch(Exception e) {
+      e.printStackTrace();
+    } finally {
+    }
+  }
+
+
+
+
   public void testDone() throws IOException {
-    close();
+      close();
   }
 
   /**
@@ -524,17 +588,19 @@
 
   public void close() {
     try {
+      int length;
 
       // print out the fuse debug output
       {
+      do {
       InputStream i = fuse_process.getInputStream();
       byte b[] = new byte[i.available()];
-      int length = i.read(b);
+      length = i.read(b);
       System.err.println("read x bytes: " + length);
       System.err.write(b,0,b.length);
+      } while(length > 0) ;
       }
 
-      int length;
       do {
       InputStream i = fuse_process.getErrorStream();
       byte b[] = new byte[i.available()];