|
@@ -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 default_id = 99; // nobody - not configurable since soon
|
|
|
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 @@ struct options {
|
|
|
int port;
|
|
|
int debug;
|
|
|
int read_only;
|
|
|
+ int initchecks;
|
|
|
int usetrash;
|
|
|
int entry_timeout;
|
|
|
int attribute_timeout;
|
|
@@ -92,6 +95,7 @@ typedef struct dfs_fh_struct {
|
|
|
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 @@ enum
|
|
|
KEY_PRIVATE,
|
|
|
KEY_BIGWRITES,
|
|
|
KEY_DEBUG,
|
|
|
+ KEY_INITCHECKS,
|
|
|
};
|
|
|
|
|
|
static struct fuse_opt dfs_opts[] =
|
|
@@ -131,6 +136,7 @@ static struct fuse_opt dfs_opts[] =
|
|
|
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 @@ int dfs_options(void *data, const char *arg, int key, struct fuse_args *outargs
|
|
|
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 @@ static int dfs_rename(const char *from, const char *to);
|
|
|
// 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 @@ int move_to_trash(const char *item) {
|
|
|
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 @@ int move_to_trash(const char *item) {
|
|
|
}
|
|
|
|
|
|
// 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 @@ int move_to_trash(const char *item) {
|
|
|
}
|
|
|
|
|
|
// 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 @@ int move_to_trash(const char *item) {
|
|
|
}
|
|
|
|
|
|
|
|
|
+/**
|
|
|
+ * 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 @@ static int dfs_getattr(const char *path, struct stat *st)
|
|
|
// 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 @@ static int dfs_getattr(const char *path, struct stat *st)
|
|
|
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 @@ static int dfs_getattr(const char *path, struct stat *st)
|
|
|
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 @@ static int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
|
|
|
(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 @@ static int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
|
|
|
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 @@ static int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
|
|
|
}
|
|
|
|
|
|
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 @@ static int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
|
|
|
|
|
|
// free the info pointers
|
|
|
hdfsFreeFileInfo(info,numEntries);
|
|
|
- syslog(LOG_ERR, "returning dfs_readdir for %s\n",path);
|
|
|
-
|
|
|
return 0;
|
|
|
}
|
|
|
|
|
@@ -508,25 +644,15 @@ static int dfs_read(const char *path, char *buf, size_t size, off_t offset,
|
|
|
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 @@ static int dfs_read(const char *path, char *buf, size_t size, off_t offset,
|
|
|
//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 @@ static int dfs_statfs(const char *path, struct statvfs *st)
|
|
|
// 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 @@ static int dfs_statfs(const char *path, struct statvfs *st)
|
|
|
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 @@ static int dfs_mkdir(const char *path, mode_t mode)
|
|
|
// 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 @@ static int dfs_mkdir(const char *path, mode_t mode)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
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 @@ static int dfs_rename(const char *from, const char *to)
|
|
|
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 @@ static int dfs_rename(const char *from, const char *to)
|
|
|
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 @@ static int dfs_rmdir(const char *path)
|
|
|
// 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 @@ static int dfs_rmdir(const char *path)
|
|
|
return -EACCES;
|
|
|
}
|
|
|
|
|
|
+ hdfsFS userFS;
|
|
|
+ // if not connected, try to connect and fail out if we can't.
|
|
|
+ if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
|
|
|
+ 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 @@ static int dfs_rmdir(const char *path)
|
|
|
}
|
|
|
|
|
|
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 @@ static int dfs_unlink(const char *path)
|
|
|
// 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 @@ static int dfs_unlink(const char *path)
|
|
|
return -EACCES;
|
|
|
}
|
|
|
|
|
|
+ hdfsFS userFS;
|
|
|
+ // if not connected, try to connect and fail out if we can't.
|
|
|
+ if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
|
|
|
+ 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 @@ static int dfs_unlink(const char *path)
|
|
|
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_unlink(const char *path)
|
|
|
|
|
|
}
|
|
|
|
|
|
+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);
|
|
|
|
|
|
-//static int dfs_truncate(const char *path, off_t size)
|
|
|
-//{
|
|
|
-// (void)path;
|
|
|
-// (void)size;
|
|
|
-// return -ENOTSUP;
|
|
|
-//}
|
|
|
+ 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;
|
|
|
+
|
|
|
+}
|
|
|
|
|
|
-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 @@ static int dfs_open(const char *path, struct fuse_file_info *fi)
|
|
|
|
|
|
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 @@ static int dfs_open(const char *path, struct fuse_file_info *fi)
|
|
|
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 @@ static int dfs_write(const char *path, const char *buf, size_t size,
|
|
|
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 @@ int dfs_release (const char *path, struct fuse_file_info *fi) {
|
|
|
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 @@ int dfs_flush(const char *path, struct fuse_file_info *fi) {
|
|
|
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 @@ int dfs_flush(const char *path, struct fuse_file_info *fi) {
|
|
|
// 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 @@ int dfs_flush(const char *path, struct fuse_file_info *fi) {
|
|
|
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;
|
|
|
|
|
|
-void dfs_setattr(struct stat *attr, int to_set, struct fuse_file_info *fi)
|
|
|
+ // 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;
|
|
|
+}
|
|
|
+
|
|
|
+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 @@ static void init_protectedpaths(dfs_context *dfs) {
|
|
|
}
|
|
|
|
|
|
|
|
|
-
|
|
|
void *dfs_init()
|
|
|
{
|
|
|
|
|
@@ -1195,6 +1312,7 @@ static struct fuse_operations dfs_oper = {
|
|
|
.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 @@ static struct fuse_operations dfs_oper = {
|
|
|
.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 @@ int main(int argc, char *argv[])
|
|
|
}
|
|
|
|
|
|
|
|
|
+ //
|
|
|
+ // 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");
|