[Darshan-commits] [Git][darshan/darshan][lustre-mod] 8 commits: new fs-specific counters for storage server and metadata server counts

Glenn K. Lockwood xgitlab at cels.anl.gov
Fri May 20 09:11:39 CDT 2016


Glenn K. Lockwood pushed to branch lustre-mod at darshan / darshan


Commits:
08956bd2 by Glenn K. Lockwood at 2016-05-18T21:38:05-07:00
new fs-specific counters for storage server and metadata server counts

- - - - -
19147a7b by Glenn K. Lockwood at 2016-05-18T21:39:00-07:00
checkpoint of code that populates OST and MDT counts per file system--doesn't seem to work though...

- - - - -
01bc23aa by Glenn K. Lockwood at 2016-05-18T21:39:48-07:00
moved the call to the lustre module up into the file open macro

- - - - -
b547fbfe by Glenn K. Lockwood at 2016-05-18T21:40:18-07:00
replaced the dummy counter with some real counters to describe file geometry on Lustre

- - - - -
64a4a226 by Glenn K. Lockwood at 2016-05-18T21:40:53-07:00
work in progress on enabling new lustre counters for files

- - - - -
4b63144a by Glenn K. Lockwood at 2016-05-19T19:38:33-07:00
fixed bug that was preventing OST/MDT counts from being populated correctly

- - - - -
08d83a03 by Glenn K. Lockwood at 2016-05-19T19:41:24-07:00
added filename hashing so multiple opens/closes don't create multiple lustre-mod records; added proper reducer so that shared files are flattened into a single set of counters whose values are dictated by rank 0

- - - - -
0934519c by Glenn K. Lockwood at 2016-05-20T07:10:58-07:00
tool to test ioctl/llapi calls, based on stat-perf.c

- - - - -


6 changed files:

- darshan-lustre-log-format.h
- darshan-runtime/darshan.h
- darshan-runtime/lib/darshan-core.c
- darshan-runtime/lib/darshan-lustre.c
- darshan-runtime/lib/darshan-posix.c
- + darshan-test/2.x/llapi-perf.c


Changes:

=====================================
darshan-lustre-log-format.h
=====================================
--- a/darshan-lustre-log-format.h
+++ b/darshan-lustre-log-format.h
@@ -12,8 +12,16 @@
 
 /* TODO: add integer counters here (e.g., counter for stripe width, stripe size, etc etc) */
 #define LUSTRE_COUNTERS \
-    /* dummy counter */\
-    X(LUSTRE_TEST_COUNTER) \
+    /* number of OSTs for file system */\
+    X(LUSTRE_OSTS) \
+    /* number of MDTs for file system */\
+    X(LUSTRE_MDTS) \
+    /* bytes per stripe for file */\
+    X(LUSTRE_STRIPE_SIZE) \
+    /* number of stripes (OSTs) for file */\
+    X(LUSTRE_STRIPE_WIDTH) \
+    /* index of first OST for file */\
+    X(LUSTRE_STRIPE_OFFSET) \
     /* end of counters */\
     X(LUSTRE_NUM_INDICES)
 


=====================================
darshan-runtime/darshan.h
=====================================
--- a/darshan-runtime/darshan.h
+++ b/darshan-runtime/darshan.h
@@ -85,6 +85,8 @@ struct darshan_fs_info
 {
     int fs_type;
     int block_size;
+    int ost_count;
+    int mdt_count;
 };
 
 /* paths that darshan will not trace */


=====================================
darshan-runtime/lib/darshan-core.c
=====================================
--- a/darshan-runtime/lib/darshan-core.c
+++ b/darshan-runtime/lib/darshan-core.c
@@ -18,6 +18,8 @@
 #include <time.h>
 #include <limits.h>
 #include <pthread.h>
+#include <dirent.h>
+#include <sys/ioctl.h>
 #include <sys/types.h>
 #include <sys/stat.h>
 #include <sys/vfs.h>
@@ -30,6 +32,9 @@
 #include "darshan-core.h"
 #include "darshan-dynamic.h"
 
+/* XXX stick this into autoconf .h */
+#include <lustre/lustreapi.h>
+
 extern char* __progname;
 extern char* __progname_full;
 
@@ -961,6 +966,34 @@ static void add_entry(char* trailing_data, int* space_left, struct mntent *entry
     else
         mnt_data_array[mnt_data_count].fs_info.block_size = 4096;
 
+    /* attempt to retrieve OST and MDS counts from Lustre */
+    mnt_data_array[mnt_data_count].fs_info.ost_count = -1;
+    mnt_data_array[mnt_data_count].fs_info.mdt_count = -1;
+    if ( statfsbuf.f_type == LL_SUPER_MAGIC )
+    {
+        int n_ost, n_mdt;
+        int ret_ost, ret_mdt;
+        DIR *mount_dir;
+
+        mount_dir = opendir( entry->mnt_dir );
+        if ( mount_dir  ) 
+        {
+            /* n_ost and n_mdt are used for both input and output to ioctl */
+            n_ost = 0;
+            n_mdt = 1;
+
+            ret_ost = ioctl( dirfd(mount_dir), LL_IOC_GETOBDCOUNT, &n_ost );
+            ret_mdt = ioctl( dirfd(mount_dir), LL_IOC_GETOBDCOUNT, &n_mdt );
+
+            if ( !(ret_ost < 0 || ret_mdt < 0) )
+            {
+                mnt_data_array[mnt_data_count].fs_info.ost_count = n_ost;
+                mnt_data_array[mnt_data_count].fs_info.mdt_count = n_mdt;
+            }
+            closedir( mount_dir );
+        }
+    }
+
     /* store mount information for use in header of darshan log */
     ret = snprintf(tmp_mnt, 256, "\n%s\t%s",
         entry->mnt_type, entry->mnt_dir);


=====================================
darshan-runtime/lib/darshan-lustre.c
=====================================
--- a/darshan-runtime/lib/darshan-lustre.c
+++ b/darshan-runtime/lib/darshan-lustre.c
@@ -16,12 +16,22 @@
 #include <stdlib.h>
 #include <assert.h>
 #include <pthread.h>
+#include <sys/ioctl.h>
+
+/* XXX stick this into autoconf .h */
+#include <lustre/lustreapi.h>
 
 #include "uthash.h"
 
 #include "darshan.h"
 #include "darshan-dynamic.h"
 
+struct lustre_record_runtime
+{
+    struct darshan_lustre_record *record;
+    UT_hash_handle hlink;
+};
+
 /* we just use a simple array for storing records. the POSIX module
  * only calls into the Lustre module for new records, so we will never
  * have to search for an existing Lustre record (assuming the Lustre
@@ -30,8 +40,10 @@
 struct lustre_runtime
 {
     struct darshan_lustre_record *record_array;
+    struct lustre_record_runtime *record_runtime_array;
     int record_array_size;
     int record_array_ndx;
+    struct lustre_record_runtime *record_hash;
 };
 
 static struct lustre_runtime *lustre_runtime = NULL;
@@ -45,52 +57,93 @@ static void lustre_begin_shutdown(void);
 static void lustre_get_output_data(MPI_Comm mod_comm, darshan_record_id *shared_recs,
     int shared_rec_count, void **lustre_buf, int *lustre_buf_sz);
 static void lustre_shutdown(void);
+static int lustre_record_compare(const void* a_p, const void* b_p);
+static void lustre_record_reduction_op(void* infile_v, void* inoutfile_v,
+    int *len, MPI_Datatype *datatype);
 
 #define LUSTRE_LOCK() pthread_mutex_lock(&lustre_runtime_mutex)
 #define LUSTRE_UNLOCK() pthread_mutex_unlock(&lustre_runtime_mutex)
 
-/* TODO: is there any way we can further compact Lustre data to save space?
- * e.g., are all files in the same directory guaranteed same striping parameters?
- * if so, can we store stripe parameters on per-directory basis and the OST
- * list on a per-file basis? maybe the storage savings are small enough this isn't
- * worth it, but nice to keep in mind
- */
-
-void darshan_instrument_lustre_file(char *filepath)
+void darshan_instrument_lustre_file(const char* filepath, int fd)
 {
-    struct darshan_lustre_record *rec;
+    struct lustre_record_runtime *rec_rt;
+    struct darshan_fs_info fs_info;
     darshan_record_id rec_id;
+    int limit_flag;
 
     LUSTRE_LOCK();
     /* make sure the lustre module is already initialized */
     lustre_runtime_initialize();
 
     /* if the array is full, we just back out */
-    if(lustre_runtime->record_array_ndx >= lustre_runtime->record_array_size)
-        return;
+    limit_flag = (lustre_runtime->record_array_ndx >= lustre_runtime->record_array_size);
 
     /* register a Lustre file record with Darshan */
+    fs_info.fs_type = -1;
     darshan_core_register_record(
         (void *)filepath,
         strlen(filepath),
         DARSHAN_LUSTRE_MOD,
         1,
-        0,
+        limit_flag,
         &rec_id,
-        NULL);
+        &fs_info);
 
     /* if record id is 0, darshan has no more memory for instrumenting */
     if(rec_id == 0)
+    {
+        LUSTRE_UNLOCK();
         return;
+    }
 
-    /* allocate a new lustre record and append it to the array */
-    rec = &(lustre_runtime->record_array[lustre_runtime->record_array_ndx++]);
-    rec->rec_id = rec_id;
-    rec->rank = my_rank;
-
-    /* TODO: gather lustre data, store in record hash */
-    /* counters in lustre_ref->record->counters */
-    rec->counters[LUSTRE_TEST_COUNTER] = 88;
+    /* search the hash table for this file record, and initialize if not found */
+    HASH_FIND(hlink, lustre_runtime->record_hash, &rec_id, sizeof(darshan_record_id), rec_rt );
+    if ( !rec_rt ) {
+        struct darshan_lustre_record *rec;
+        struct lov_user_md *lum;
+        size_t lumsize = sizeof(struct lov_user_md) +
+            LOV_MAX_STRIPE_COUNT * sizeof(struct lov_user_ost_data);
+
+        /* allocate a new lustre record and append it to the array */
+        rec_rt = &(lustre_runtime->record_runtime_array[lustre_runtime->record_array_ndx]);
+        rec_rt->record = &(lustre_runtime->record_array[lustre_runtime->record_array_ndx]);
+        rec = rec_rt->record;
+        rec->rec_id = rec_id;
+        rec->rank = my_rank;
+
+        /* implicit assumption here that none of these counters will change
+         * after the first time a file is opened.  This may not always be
+         * true in the future */
+        if ( fs_info.fs_type != -1 ) 
+        {
+            rec->counters[LUSTRE_OSTS] = fs_info.ost_count;
+            rec->counters[LUSTRE_MDTS] = fs_info.mdt_count;
+        }
+        else
+        {
+            rec->counters[LUSTRE_OSTS] = -1;
+            rec->counters[LUSTRE_MDTS] = -1;
+        }
+
+        if ( (lum = calloc(1, lumsize)) != NULL ) {
+            lum->lmm_magic = LOV_USER_MAGIC;
+            /* don't care about the return code for ioctl */
+            ioctl( fd, LL_IOC_LOV_GETSTRIPE, (void *)lum );
+            rec->counters[LUSTRE_STRIPE_SIZE] = lum->lmm_stripe_size;
+            rec->counters[LUSTRE_STRIPE_WIDTH] = lum->lmm_stripe_count;
+            rec->counters[LUSTRE_STRIPE_OFFSET] = 0; /* this currently doesn't work; lum->lmm_objects[0].l_ost_idx isn't being populated */
+            /* TODO: add explicit list of OSTs */
+            free(lum);
+        }
+        else
+        {
+            rec->counters[LUSTRE_STRIPE_SIZE] = -1;
+            rec->counters[LUSTRE_STRIPE_WIDTH] = -1;
+            rec->counters[LUSTRE_STRIPE_OFFSET] = -1;
+        }
+        HASH_ADD(hlink, lustre_runtime->record_hash, record->rec_id, sizeof(darshan_record_id), rec_rt);
+        lustre_runtime->record_array_ndx++;
+    }
 
     LUSTRE_UNLOCK();
     return;
@@ -142,6 +195,16 @@ static void lustre_runtime_initialize()
     memset(lustre_runtime->record_array, 0, lustre_runtime->record_array_size *
         sizeof(struct darshan_lustre_record));
 
+    lustre_runtime->record_runtime_array = malloc(lustre_runtime->record_array_size *
+                                          sizeof(struct lustre_record_runtime));
+    if(!lustre_runtime->record_runtime_array)
+    {
+        lustre_runtime->record_array_size = 0;
+        return;
+    }
+    memset(lustre_runtime->record_runtime_array, 0, lustre_runtime->record_array_size *
+        sizeof(struct lustre_record_runtime));
+
     return;
 }
 
@@ -168,13 +231,75 @@ static void lustre_get_output_data(
     void **lustre_buf,
     int *lustre_buf_sz)
 {
+    struct lustre_record_runtime *file;
+    int i;
+    struct darshan_lustre_record *red_send_buf = NULL;
+    struct darshan_lustre_record *red_recv_buf = NULL;
+    MPI_Datatype red_type;
+    MPI_Op red_op;
+
     assert(lustre_runtime);
 
-    /* TODO: determine lustre record shared across all processes,
-     * and have only rank 0 write these records out. No shared 
-     * reductions should be necessary as the Lustre data for a
-     * given file should be the same on each process
+    /* if there are globally shared files, do a shared file reduction */
+    /* NOTE: the shared file reduction is also skipped if the 
+     * DARSHAN_DISABLE_SHARED_REDUCTION environment variable is set.
      */
+    if(shared_rec_count && !getenv("DARSHAN_DISABLE_SHARED_REDUCTION"))
+    {
+        /* necessary initialization of shared records */
+        for(i = 0; i < shared_rec_count; i++)
+        {
+            HASH_FIND(hlink, lustre_runtime->record_hash, &shared_recs[i],
+                sizeof(darshan_record_id), file);
+            assert(file);
+
+            file->record->rank = -1;
+        }
+
+        /* sort the array of files descending by rank so that we get all of the 
+         * shared files (marked by rank -1) in a contiguous portion at end 
+         * of the array
+         */
+        qsort(lustre_runtime->record_array, lustre_runtime->record_array_ndx,
+            sizeof(struct darshan_lustre_record), lustre_record_compare);
+
+        /* make *send_buf point to the shared files at the end of sorted array */
+        red_send_buf =
+            &(lustre_runtime->record_array[lustre_runtime->record_array_ndx-shared_rec_count]);
+
+        /* allocate memory for the reduction output on rank 0 */
+        if(my_rank == 0)
+        {
+            red_recv_buf = malloc(shared_rec_count * sizeof(struct darshan_lustre_record));
+            if(!red_recv_buf)
+            {
+                return;
+            }
+        }
+
+        DARSHAN_MPI_CALL(PMPI_Type_contiguous)(sizeof(struct darshan_lustre_record),
+            MPI_BYTE, &red_type);
+        DARSHAN_MPI_CALL(PMPI_Type_commit)(&red_type);
+        DARSHAN_MPI_CALL(PMPI_Op_create)(lustre_record_reduction_op, 1, &red_op);
+        DARSHAN_MPI_CALL(PMPI_Reduce)(red_send_buf, red_recv_buf,
+            shared_rec_count, red_type, red_op, 0, mod_comm);
+
+        /* clean up reduction state */
+        if(my_rank == 0)
+        {
+            int tmp_ndx = lustre_runtime->record_array_ndx - shared_rec_count;
+            memcpy(&(lustre_runtime->record_array[tmp_ndx]), red_recv_buf,
+                shared_rec_count * sizeof(struct darshan_lustre_record));
+            free(red_recv_buf);
+        }
+        else
+        {
+            lustre_runtime->record_array_ndx -= shared_rec_count;
+        }
+
+        DARSHAN_MPI_CALL(PMPI_Type_free)(&red_type);
+        DARSHAN_MPI_CALL(PMPI_Op_free)(&red_op);
+    }
 
     *lustre_buf = (void *)(lustre_runtime->record_array);
     *lustre_buf_sz = lustre_runtime->record_array_ndx * sizeof(struct darshan_lustre_record);
@@ -186,14 +311,68 @@ static void lustre_shutdown(void)
 {
     assert(lustre_runtime);
 
-    /* TODO: free data structures */
+    HASH_CLEAR(hlink, lustre_runtime->record_hash);
     free(lustre_runtime->record_array);
+    free(lustre_runtime->record_runtime_array);
     free(lustre_runtime);
     lustre_runtime = NULL;
 
     return;
 }
 
+/* compare function for sorting file records by descending rank */
+static int lustre_record_compare(const void* a_p, const void* b_p)
+{
+    const struct darshan_lustre_record* a = a_p;
+    const struct darshan_lustre_record* b = b_p;
+
+    if(a->rank < b->rank)
+        return 1;
+    if(a->rank > b->rank)
+        return -1;
+
+    return 0;
+}
+
+/* this is just boilerplate reduction code that isn't currently used */
+static void lustre_record_reduction_op(void* infile_v, void* inoutfile_v,
+    int *len, MPI_Datatype *datatype)
+{
+    struct darshan_lustre_record tmp_record;
+    struct darshan_lustre_record *infile = infile_v;
+    struct darshan_lustre_record *inoutfile = inoutfile_v;
+    int i, j;
+
+    assert(lustre_runtime);
+
+    for( i=0; i<*len; i++ )
+    {
+        memset(&tmp_record, 0, sizeof(struct darshan_lustre_record));
+        tmp_record.rec_id = infile->rec_id;
+        tmp_record.rank = -1;
+
+        /* preserve only rank 0's value */
+        for( j = LUSTRE_OSTS; j < LUSTRE_NUM_INDICES; j++)
+        {
+            if ( my_rank == 0 ) 
+            {
+                tmp_record.counters[j] = infile->counters[j];
+            }
+            else
+            {
+                tmp_record.counters[j] = inoutfile->counters[j];
+            }
+        }
+
+        /* update pointers */
+        *inoutfile = tmp_record;
+        inoutfile++;
+        infile++;
+    }
+
+    return;
+}
+
 /*
  * Local variables:
  *  c-indent-level: 4


=====================================
darshan-runtime/lib/darshan-posix.c
=====================================
--- a/darshan-runtime/lib/darshan-posix.c
+++ b/darshan-runtime/lib/darshan-posix.c
@@ -39,6 +39,10 @@ typedef int64_t off64_t;
 #define aiocb64 aiocb
 #endif
 
+#ifndef LL_SUPER_MAGIC
+#define LL_SUPER_MAGIC 0x0BD00BD0
+#endif
+
 DARSHAN_FORWARD_DECL(open, int, (const char *path, int flags, ...));
 DARSHAN_FORWARD_DECL(open64, int, (const char *path, int flags, ...));
 DARSHAN_FORWARD_DECL(creat, int, (const char* path, mode_t mode));
@@ -128,6 +132,7 @@ struct posix_file_runtime
     int access_count;
     void* stride_root;
     int stride_count;
+    int fs_type; /* same as darshan_fs_info->fs_type */
     struct posix_aio_tracker* aio_list;
     UT_hash_handle hlink;
 };
@@ -179,7 +184,7 @@ struct posix_runtime
 /* XXX modules don't expose an API for other modules, so use extern to get
  * Lustre instrumentation function
  */
-extern void darshan_instrument_lustre_file(char *filepath);
+extern void darshan_instrument_lustre_file(const char *filepath, int fd);
 
 static struct posix_runtime *posix_runtime = NULL;
 static pthread_mutex_t posix_runtime_mutex = PTHREAD_RECURSIVE_MUTEX_INITIALIZER_NP;
@@ -234,6 +239,8 @@ static void posix_shutdown(void);
     if(file->file_record->fcounters[POSIX_F_OPEN_TIMESTAMP] == 0 || \
      file->file_record->fcounters[POSIX_F_OPEN_TIMESTAMP] > __tm1) \
         file->file_record->fcounters[POSIX_F_OPEN_TIMESTAMP] = __tm1; \
+    if(file->fs_type == LL_SUPER_MAGIC) \
+        darshan_instrument_lustre_file(__path, __ret); \
     DARSHAN_TIMER_INC_NO_OVERLAP(file->file_record->fcounters[POSIX_F_META_TIME], __tm1, __tm2, file->last_meta_end); \
 } while(0)
 
@@ -1561,18 +1568,12 @@ static struct posix_file_runtime* posix_file_by_name(const char *name)
     file->file_record->rank = my_rank;
     file->file_record->counters[POSIX_MEM_ALIGNMENT] = darshan_mem_alignment;
     file->file_record->counters[POSIX_FILE_ALIGNMENT] = fs_info.block_size;
+    file->fs_type = fs_info.fs_type;
 
     /* add new record to file hash table */
     HASH_ADD(hlink, posix_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
     posix_runtime->file_array_ndx++;
 
-    /* XXX: check for lustre and call in */
-#ifndef LL_SUPER_MAGIC
-#define LL_SUPER_MAGIC 0x0BD00BD0
-#endif
-    if(fs_info.fs_type == LL_SUPER_MAGIC)
-        darshan_instrument_lustre_file(newname);
-
     if(newname != name)
         free(newname);
     return(file);


=====================================
darshan-test/2.x/llapi-perf.c
=====================================
--- /dev/null
+++ b/darshan-test/2.x/llapi-perf.c
@@ -0,0 +1,313 @@
+/*
+ *  (C) 2012 by Argonne National Laboratory.
+ *      See COPYRIGHT in top-level directory.
+ */
+
+/* llapi-perf.c
+ * Time how long it takes to extract various file data from Lustre via
+ * ioctl and llapi calls from every process.  -i uses ioctl, -a uses the
+ * Lustre API.  This also retains the features of stat-perf.c, which
+ * times how long it takes to issue a stat64() call to the designated file
+ * from every process.  -f causes it to use fstat64() rather than stat64().  
+ * -l causes it to use lseek(SEEK_END) instead of stat64().
+ * -c causes it to create the file from scratch rather than operating on an
+ *  existing file.  -r issues a realpath() call on the file.
+ */
+
+#define _LARGEFILE64_SOURCE
+
+#include <assert.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <fcntl.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <string.h>
+#include <sys/time.h>
+#include <mpi.h>
+#include <errno.h>
+#include <getopt.h>
+#include <sys/ioctl.h>
+#ifndef NO_LUSTRE
+#include <lustre/lustreapi.h>
+#endif
+
+static char* opt_file = NULL;
+static int opt_create = 0;
+static int opt_fstat = 0;
+static int opt_lseek = 0;
+static int opt_realpath = 0;
+static int opt_ioctl = 0;
+static int opt_llapi = 0;
+static int opt_fpp = 0;
+static int rank = -1;
+
+static int parse_args(int argc, char **argv);
+static void usage(void);
+
+int main(int argc, char **argv)
+{
+   int fd;
+   int ret;
+   double stime, etime, elapsed, slowest;
+   struct stat64 statbuf;
+   int nprocs;
+   off64_t offset, orig_offset;
+   char* new_path;
+
+   MPI_Init(&argc,&argv);
+   MPI_Comm_rank(MPI_COMM_WORLD, &rank);
+   MPI_Comm_size(MPI_COMM_WORLD, &nprocs);
+   
+   /* parse the command line arguments */
+   parse_args(argc, argv);
+
+   MPI_Barrier(MPI_COMM_WORLD);
+
+   /* open specified file */
+   if(!opt_create)
+   {
+      fd = open(opt_file, O_RDWR);  
+      if(fd < 0)
+      {
+         perror("open");
+         exit(1);
+      }
+   }
+   else
+   {
+      /* rank 0 create, everyone else open */
+      if(rank == 0 || opt_fpp)
+      {
+         fd = open(opt_file, O_RDWR|O_CREAT|O_EXCL, S_IRUSR|S_IWUSR);
+         if(fd < 0)
+         {
+            perror("open");
+            exit(1);
+         }
+         MPI_Barrier(MPI_COMM_WORLD);
+      }
+      else
+      {
+         MPI_Barrier(MPI_COMM_WORLD);
+         fd = open(opt_file, O_RDWR);  
+         if(fd < 0)
+         {
+            perror("open");
+            exit(1);
+         }
+      }
+   }
+
+   MPI_Barrier(MPI_COMM_WORLD);
+   stime = MPI_Wtime();
+
+   ret = 0;
+   if(opt_fstat)
+      ret = fstat64(fd, &statbuf);
+   else if(opt_lseek)
+   {
+      /* find current position */
+      orig_offset = lseek64(fd, 0, SEEK_CUR);
+      if(orig_offset < 0)
+         ret = -1;
+      else
+      {
+         /* find end of file; this is the size */
+         offset = lseek64(fd, 0, SEEK_END);
+         if(offset < 0)
+            ret = -1;
+         else
+         {
+            /* go back to original position */
+            offset = lseek64(fd, orig_offset, SEEK_SET);
+            if(offset < 0)
+                ret = -1;
+         }
+      }
+   }
+   else if(opt_realpath)
+   {
+      new_path = realpath(opt_file, NULL);
+      if(!new_path)
+        ret = -1;
+      else
+        free(new_path);
+   }
+   else if ( opt_llapi || opt_ioctl )
+   {
+#ifdef NO_LUSTRE
+      fprintf(stderr, "Not compiled with Lustre support\n");
+      ret = -1;
+#else
+      struct lov_user_md *lum;
+      size_t lumsize = sizeof(struct lov_user_md) +
+           LOV_MAX_STRIPE_COUNT * sizeof(struct lov_user_ost_data);
+
+      lum = calloc(1, lumsize);
+      if (lum == NULL) {
+         ret = ENOMEM;
+         fprintf(stderr, "No memory\n");
+      }
+      else {
+        if ( opt_llapi ) 
+        {
+         ret = llapi_file_get_stripe(opt_file, lum);
+        }
+        else if ( opt_ioctl )
+        {
+            lum->lmm_magic = LOV_USER_MAGIC;
+            ret = ioctl( fd, LL_IOC_LOV_GETSTRIPE, (void *)lum );
+        }
+#ifdef DEBUG
+        /* different API/ioctl calls populate only parts of lum */
+        printf( "stripe_width=%d stripe_size=%d starting_ost=%d\n",
+             lum->lmm_stripe_count,
+             lum->lmm_stripe_size,
+             lum->lmm_stripe_count );
+#endif
+        }
+#endif
+   }
+   else
+      ret = stat64(opt_file, &statbuf);
+
+   if(ret != 0 && !opt_ioctl && !opt_llapi)
+   {
+      perror("stat64 or fstat64");
+      exit(1);
+   }
+#ifndef NO_LUSTRE
+   else if ( ret < 0 && opt_ioctl )
+   {
+      perror("ioctl");
+      exit(1);
+   }
+#endif
+   
+   etime = MPI_Wtime();
+
+   elapsed = etime-stime;
+   ret = MPI_Reduce(&elapsed, &slowest, 1, MPI_DOUBLE, MPI_MAX, 0, MPI_COMM_WORLD);
+   if(ret != 0)
+   {
+      fprintf(stderr, "Error: MPI_Reduce() failure.\n");
+      exit(1);
+   }
+
+
+   slowest *= 1000.0;
+
+   if(rank == 0)
+   {
+      printf("opt_file: %s, opt_create: %d, opt_fstat: %d, opt_lseek: %d, opt_realpath: %d, opt_llapi: %d, opt_ioctl: %d, opt_fpp: %d, nprocs: %d, time: %f ms\n",
+        opt_file,
+        opt_create,
+        opt_fstat,
+        opt_lseek,
+        opt_realpath,
+        opt_llapi,
+        opt_ioctl,
+        opt_fpp,
+        nprocs,
+        slowest);
+   }
+
+   MPI_Finalize();
+   return(0);
+}
+
+static int parse_args(int argc, char **argv)
+{
+   int c;
+   
+   while ((c = getopt(argc, argv, "fclripa")) != EOF) {
+      switch (c) {
+         case 'c': /* create file */
+            opt_create = 1;
+            break;
+         case 'f': /* fstat instead of stat */
+            opt_fstat = 1;
+            break;
+         case 'l': /* lseek instead of stat */
+            opt_lseek = 1;
+            break;
+         case 'r': /* realpath instead of stat */
+            opt_realpath = 1;
+            break;
+         case 'i': /* use ioctl test */
+            opt_ioctl = 1;
+            break;
+         case 'a': /* use llapi test*/
+            opt_llapi = 1;
+            break;
+         case 'p': /* file per process instead of shared file */
+            opt_fpp = 1;
+            break;
+         case 'h':
+            if (rank == 0)
+                usage();
+            exit(0);
+         case '?': /* unknown */
+            if (rank == 0)
+                usage();
+            exit(1);
+         default:
+            break;
+      }
+   }
+
+   if(opt_lseek + opt_fstat + opt_realpath + opt_ioctl + opt_llapi > 1)
+   {
+      fprintf(stderr, "Error: Only specify one of -l, -f, -i, -a, or -r.\n");
+      usage();
+      exit(1);
+   }
+
+   if(argc-optind != 1)
+   {
+      if(rank == 0)
+          usage();
+      exit(1);
+   }
+
+   if ( opt_fpp ) 
+   {
+      opt_file = malloc( sizeof(char) * (strlen( argv[optind] ) + 10) );
+      sprintf( opt_file, "%s.%d", argv[optind], rank );
+   }
+   else 
+   {
+      opt_file = strdup(argv[optind]);
+   }
+   assert(opt_file);
+
+   return(0);
+}
+
+static void usage(void)
+{
+    printf("Usage: stat-perf [<OPTIONS>...] <FILE NAME>\n");
+    printf("\n<OPTIONS> is one or more of\n");
+    printf(" -c       create new file to stat\n");
+    printf(" -p       do file-per-process instead of shared file\n");
+    printf(" -f       use fstat instead of stat\n");
+    printf(" -l       use lseek instead of stat\n");
+    printf(" -r       use realpath instead of stat\n");
+    printf(" -a       use Lustre API test\n");
+    printf(" -i       use ioctl Lustre test\n");
+    printf(" -h       print this help\n");
+}
+
+/*
+ * Local variables:
+ *  c-indent-level: 3
+ *  c-basic-offset: 3
+ *  tab-width: 3
+ *
+ * vim: ts=3
+ * End:
+ */ 
+
+



View it on GitLab: https://xgitlab.cels.anl.gov/darshan/darshan/compare/3266825fa44dbc8e1cefbe99d189ae491d1ed42c...0934519c3a554a90443917e93ba0fb367aca6047
-------------- next part --------------
An HTML attachment was scrubbed...
URL: <http://lists.mcs.anl.gov/pipermail/darshan-commits/attachments/20160520/260127ab/attachment-0001.html>


More information about the Darshan-commits mailing list