[Darshan-commits] [Git][darshan/darshan][mmap-dev] darshan-core code and documentation cleanup

Shane Snyder xgitlab at cels.anl.gov
Wed Jun 8 17:04:20 CDT 2016


Shane Snyder pushed to branch mmap-dev at darshan / darshan


Commits:
d8f3575a by Shane Snyder at 2016-06-08T16:58:23-05:00
darshan-core code and documentation cleanup

Cleanup the structure and documenation of the code to be in sync
with the new mmap version of Darshan. In particular, I made sure
all extra mmap logic was properly ifdef wrapped so it's more clear
which portions of the code are responsible for implementing that
feature.

- - - - -


3 changed files:

- darshan-runtime/darshan-core.h
- darshan-runtime/darshan.h
- darshan-runtime/lib/darshan-core.c


Changes:

=====================================
darshan-runtime/darshan-core.h
=====================================
--- a/darshan-runtime/darshan-core.h
+++ b/darshan-runtime/darshan-core.h
@@ -58,6 +58,7 @@ struct darshan_core_module
     struct darshan_module_funcs funcs;
 };
 
+/* strucutre for keeping a reference to registered name records */
 struct darshan_core_name_record_ref
 {
     struct darshan_name_record *name_record;
@@ -80,6 +81,7 @@ struct darshan_core_runtime
     struct darshan_core_module* mod_array[DARSHAN_MAX_MODS];
     int mod_mem_used;
     struct darshan_core_name_record_ref *name_hash;
+    int name_mem_used; 
     double wtime_offset;
     char *comp_buf;
 #ifdef __DARSHAN_ENABLE_MMAP_LOGS


=====================================
darshan-runtime/darshan.h
=====================================
--- a/darshan-runtime/darshan.h
+++ b/darshan-runtime/darshan.h
@@ -92,11 +92,14 @@ struct darshan_module_funcs
  * Register module identifier 'mod_id' with the darshan-core runtime
  * environment, allowing the module to store I/O characterization data.
  * 'funcs' is a pointer to a structure containing each of the function
- * pointers required by darshan-core to shut down the module. The function
- * returns the following integers passed in as pointers: 'rank' is the
- * MPI rank of the calling process, 'mod_mem_limit' is the maximum amount
- * of memory the module may use, and 'sys_mem_alignment' is the configured
- * memory alignment value Darshan was configured with.
+ * pointers required by darshan-core to shut down the module.
+ * 'inout_mod_buf_size' is an input/output argument, with it being
+ * set to the requested amount of module memory on input, and set to
+ * the amount allocated by darshan-core on output. If given, 'rank' is
+ * a pointer to an integer which will contain the calling process's
+ * MPI rank on return. If given, 'sys_mem_alignment' is a pointer to
+ * an integer which will contain the memory alignment value Darshan
+ * was configured with on return.
  */
 void darshan_core_register_module(
     darshan_module_id mod_id,
@@ -115,7 +118,7 @@ void darshan_core_unregister_module(
 
 /* darshan_core_gen_record_id()
  *
- *
+ * Returns the Darshan record ID correpsonding to input string 'name'.
  */
 darshan_record_id darshan_core_gen_record_id(
     const char *name);
@@ -124,15 +127,14 @@ darshan_record_id darshan_core_gen_record_id(
  *
  * Register a record with the darshan-core runtime, allowing it to be
  * properly tracked and (potentially) correlated with records from other
- * modules. 'name' is the the name of the Darshan record (e.g., the full
- * file path) and 'len' is the size of the name pointer (string length
- * for string names). 'mod_id' is the identifier of the calling module,
- * 'printable_flag' indicates whether the name is a string, and 
- * 'mod_limit_flag' is set if the calling module is out of memory (to
- * prevent darshan-core from creating new records and to just search
- * through existing records). 'rec_id' is an output pointer storing the
- * correspoing Darshan record identifier and 'file_alignment' is an output
- * pointer storing the file system alignment value for the given record.
+ * modules. 'rec_id' is the Darshan record id as given by the
+ * `darshan_core_gen_record_id` function. 'name' is the the name of the
+ * Darshan record (e.g., the full file path), which for now is just a
+ * string. 'mod_id' is the identifier of the calling module. 'rec_len'
+ * is the size of the record being registered with Darshan. If given,
+ * 'file_alignment' is a pointer to an integer which on return will
+ * contain the corresponding file system alignment of the file system
+ * path 'name' resides on.
  */
 void *darshan_core_register_record(
     darshan_record_id rec_id,
@@ -150,8 +152,8 @@ double darshan_core_wtime(void);
 
 /* darshan_core_excluded_path()
  *
- * Returns true (1) if the given file path is in Darshan's list of
- * excluded file paths, false (0) otherwise.
+ * Returns true (1) if the given file path 'path' is in Darshan's
+ * list of excluded file paths, false (0) otherwise.
  */
 int darshan_core_excluded_path(
     const char * path);


=====================================
darshan-runtime/lib/darshan-core.c
=====================================
--- a/darshan-runtime/lib/darshan-core.c
+++ b/darshan-runtime/lib/darshan-core.c
@@ -41,6 +41,7 @@ static pthread_mutex_t darshan_core_mutex = PTHREAD_RECURSIVE_MUTEX_INITIALIZER_
 static int my_rank = -1;
 static int nprocs = -1;
 static int darshan_mem_alignment = 1;
+static long darshan_mod_mem_quota = DARSHAN_MOD_MEM_MAX;
 
 /* paths prefixed with the following directories are not traced by darshan */
 char* darshan_path_exclusions[] = {
@@ -89,11 +90,13 @@ static struct mnt_data mnt_data_array[DARSHAN_MAX_MNTS];
 static int mnt_data_count = 0;
 
 /* prototypes for internal helper functions */
+static void *darshan_init_mmap_log(
+    struct darshan_core_runtime* core, int jobid);
 static void darshan_log_record_hints_and_ver(
     struct darshan_core_runtime* core);
 static void darshan_get_exe_and_mounts(
     struct darshan_core_runtime *core, int argc, char **argv);
-static void darshan_add_name_record_ref(
+static int darshan_add_name_record_ref(
     struct darshan_core_runtime *core, darshan_record_id rec_id,
     const char *name, darshan_module_id mod_id);
 static int darshan_block_size_from_path(
@@ -132,13 +135,6 @@ void darshan_core_initialize(int argc, char **argv)
     int ret;
     int tmpval;
     int i;
-#ifdef __DARSHAN_ENABLE_MMAP_LOGS
-    int mmap_fd;
-    int mmap_size;
-    int sys_page_size;
-    char cuser[L_cuserid] = {0};
-    char *mmap_log_dir;
-#endif
 
     DARSHAN_MPI_CALL(PMPI_Comm_size)(MPI_COMM_WORLD, &nprocs);
     DARSHAN_MPI_CALL(PMPI_Comm_rank)(MPI_COMM_WORLD, &my_rank);
@@ -203,13 +199,15 @@ void darshan_core_initialize(int argc, char **argv)
             memset(init_core, 0, sizeof(*init_core));
             init_core->wtime_offset = DARSHAN_MPI_CALL(PMPI_Wtime)();
 
+        /* TODO: do we alloc new memory as we go or just do everything up front? */
+
 #ifndef __DARSHAN_ENABLE_MMAP_LOGS
             /* just allocate memory for each log file region */
             init_core->log_hdr_p = malloc(sizeof(struct darshan_header));
             init_core->log_job_p = malloc(sizeof(struct darshan_job));
             init_core->log_exemnt_p = malloc(DARSHAN_EXE_LEN+1);
             init_core->log_name_p = malloc(DARSHAN_NAME_RECORD_BUF_SIZE);
-            init_core->log_mod_p = malloc(DARSHAN_MOD_MEM_MAX);
+            init_core->log_mod_p = malloc(darshan_mod_mem_quota);
 
             if(!(init_core->log_hdr_p) || !(init_core->log_job_p) ||
                !(init_core->log_exemnt_p) || !(init_core->log_name_p) ||
@@ -223,75 +221,18 @@ void darshan_core_initialize(int argc, char **argv)
             memset(init_core->log_job_p, 0, sizeof(struct darshan_job));
             memset(init_core->log_exemnt_p, 0, DARSHAN_EXE_LEN+1);
             memset(init_core->log_name_p, 0, DARSHAN_NAME_RECORD_BUF_SIZE);
-            memset(init_core->log_mod_p, 0, DARSHAN_MOD_MEM_MAX);
+            memset(init_core->log_mod_p, 0, darshan_mod_mem_quota);
 #else
             /* if mmap logs are enabled, we need to initialize the mmap region
              * before setting the corresponding log file region pointers
              */
-
-            sys_page_size = sysconf(_SC_PAGESIZE);
-            assert(sys_page_size > 0);
-
-            mmap_size = sizeof(struct darshan_header) + DARSHAN_JOB_RECORD_SIZE +
-                + DARSHAN_NAME_RECORD_BUF_SIZE + DARSHAN_MOD_MEM_MAX;
-            if(mmap_size % sys_page_size)
-                mmap_size = ((mmap_size / sys_page_size) + 1) * sys_page_size;
-
-            envstr = getenv(DARSHAN_MMAP_LOG_PATH_OVERRIDE);
-            if(envstr)
-                mmap_log_dir = envstr;
-            else
-                mmap_log_dir = DARSHAN_DEF_MMAP_LOG_PATH;
-
-            darshan_get_user_name(cuser);
-
-            /* construct a unique temporary log file name for this process
-             * to write mmap log data to
-             */
-            snprintf(init_core->mmap_log_name, PATH_MAX,
-                "/%s/%s_%s_id%d_mmap-log-%d.darshan",
-                mmap_log_dir, cuser, __progname, jobid, my_rank);
-
-            /* create the temporary mmapped darshan log */
-            mmap_fd = open(init_core->mmap_log_name, O_CREAT|O_RDWR|O_EXCL , 0644);
-            if(mmap_fd < 0)
-            {
-                fprintf(stderr, "darshan library warning: "
-                    "unable to create darshan log file %s\n", init_core->mmap_log_name);
-                free(init_core);
-                return;
-            }
-
-            /* TODO: what's more expensive? truncate or write zeros? perf test this call and later accesses */
-            /* allocate the necessary space in the log file */
-            ret = ftruncate(mmap_fd, mmap_size);
-            if(ret < 0)
-            {
-                fprintf(stderr, "darshan library warning: "
-                    "unable to allocate darshan log file %s\n", init_core->mmap_log_name);
-                free(init_core);
-                close(mmap_fd);
-                unlink(init_core->mmap_log_name);
-                return;
-            }
-
-            /* memory map buffers for getting at least some summary i/o data
-             * into a log file if darshan does not shut down properly
-             */
-            void *mmap_p = mmap(NULL, mmap_size, PROT_WRITE, MAP_SHARED, mmap_fd, 0);
-            if(mmap_p == MAP_FAILED)
+            void *mmap_p = darshan_init_mmap_log(init_core, jobid);
+            if(!mmap_p)
             {
-                fprintf(stderr, "darshan library warning: "
-                    "unable to mmap darshan log file %s\n", init_core->mmap_log_name);
                 free(init_core);
-                close(mmap_fd);
-                unlink(init_core->mmap_log_name);
                 return;
             }
 
-            /* close darshan log file (this does *not* unmap the log file) */
-            close(mmap_fd);
-
             /* set the memory pointers for each log file region */
             init_core->log_hdr_p = (struct darshan_header *)mmap_p;
             init_core->log_job_p = (struct darshan_job *)
@@ -306,7 +247,7 @@ void darshan_core_initialize(int argc, char **argv)
             /* set header fields needed for the mmap log mechanism */
             init_core->log_hdr_p->comp_type = DARSHAN_NO_COMP;
             init_core->log_hdr_p->name_map.off =
-                sizeof(struct darshan_header) + DARSHAN_JOB_RECORD_SIZE;
+                ((char *)init_core->log_name_p - (char *)init_core->log_hdr_p);
 #endif
 
             /* set known header fields for the log file */
@@ -339,10 +280,6 @@ void darshan_core_initialize(int argc, char **argv)
         }
     }
 
-    /* reduce so all ranks agree on the job start time */
-    DARSHAN_MPI_CALL(PMPI_Allreduce)(MPI_IN_PLACE, &init_core->log_job_p->start_time,
-        1, MPI_INT64_T, MPI_MIN, MPI_COMM_WORLD);
-
     if(internal_timing_flag)
     {
         init_time = DARSHAN_MPI_CALL(PMPI_Wtime)() - init_start;
@@ -361,11 +298,10 @@ void darshan_core_initialize(int argc, char **argv)
 void darshan_core_shutdown()
 {
     struct darshan_core_runtime *final_core;
-    struct darshan_header out_header;
-    struct darshan_job out_job;
     int internal_timing_flag = 0;
     struct tm *start_tm;
     time_t start_time_tmp;
+    int64_t first_start_time;
     int64_t last_end_time;
     double start_log_time;
     double tm_end;
@@ -404,40 +340,27 @@ void darshan_core_shutdown()
     darshan_core = NULL;
     DARSHAN_CORE_UNLOCK();
 
-    memcpy(&out_job, final_core->log_job_p, sizeof(struct darshan_job));
-
 #ifdef __DARSHAN_ENABLE_MMAP_LOGS
-    /* TODO: can we get rid of out_ header and job?? */
-    /* XXX just copy mmap files somewhere else to avoid corruption */
-    DARSHAN_MPI_CALL(PMPI_Barrier)(MPI_COMM_WORLD);
-    char cp_cmd[500] = {0};
-    sprintf(cp_cmd, "cp %s ~/Desktop", final_core->mmap_log_name);
-    system(cp_cmd);
-    DARSHAN_MPI_CALL(PMPI_Barrier)(MPI_COMM_WORLD);
-
-    /* indicate in the metadata field of the temporary darshan log file that
-     * the darshan shutdown process was invoked on the data in the log. since
-     * we have no way of knowing how far the shutdown process got, the data
-     * could potentially be in inconsistent or corrupt state. in this case,
-     * darshan currently does not generate a log file at all.
+    /* remove the temporary mmap log files */
+    /* NOTE: this unlink is not immediate as it must wait for the mapping
+     * to no longer be referenced, which in our case happens when the
+     * executable exits. If the application terminates mid-shutdown, then
+     * there will be no mmap files and no final log file.
      */
-    char *m = final_core->log_job_p->metadata + strlen(final_core->log_job_p->metadata);
-    int meta_remain = DARSHAN_JOB_METADATA_LEN - strlen(final_core->log_job_p->metadata) - 1;
-    /* TODO: do we ever check for darshan_shutdown? */
-    snprintf(m, meta_remain, "darshan_shutdown=yes\n");
+    unlink(final_core->mmap_log_name);
 #endif
 
-    /* we also need to set which modules were registered on this process and
-     * call into those modules and give them a chance to perform any necessary
-     * pre-shutdown steps.
-     */
-    for(i = 0; i < DARSHAN_MAX_MODS; i++)
+    final_core->log_job_p->end_time = time(NULL);
+
+    /* reduce to report first start and last end time across all ranks at rank 0 */
+    DARSHAN_MPI_CALL(PMPI_Reduce)(&final_core->log_job_p->start_time, &first_start_time,
+        1, MPI_INT64_T, MPI_MIN, 0, MPI_COMM_WORLD);
+    DARSHAN_MPI_CALL(PMPI_Reduce)(&final_core->log_job_p->end_time, &last_end_time,
+        1, MPI_INT64_T, MPI_MAX, 0, MPI_COMM_WORLD);
+    if(my_rank == 0)
     {
-        if(final_core->mod_array[i])
-        {
-            local_mod_use[i] = 1;
-            final_core->mod_array[i]->funcs.begin_shutdown();
-        }
+        final_core->log_job_p->start_time = first_start_time;
+        final_core->log_job_p->end_time = last_end_time;
     }
 
     final_core->comp_buf = malloc(DARSHAN_COMP_BUF_SIZE);
@@ -476,14 +399,17 @@ void darshan_core_shutdown()
         return;
     }
 
-    final_core->log_job_p->end_time = time(NULL);
-
-    /* reduce to report last end time across all ranks at rank 0 */
-    DARSHAN_MPI_CALL(PMPI_Reduce)(&final_core->log_job_p->end_time, &last_end_time,
-        1, MPI_INT64_T, MPI_MAX, 0, MPI_COMM_WORLD);
-    if(my_rank == 0)
+    /* set which modules were registered locally, and call into
+     * them to disable further instrumentation and to perform any
+     * other pre-shutdown steps
+     */
+    for(i = 0; i < DARSHAN_MAX_MODS; i++)
     {
-        out_job.end_time = last_end_time;
+        if(final_core->mod_array[i])
+        {
+            local_mod_use[i] = 1;
+            final_core->mod_array[i]->funcs.begin_shutdown();
+        }
     }
 
     /* reduce the number of times a module was opened globally and bcast to everyone */
@@ -520,7 +446,7 @@ void darshan_core_shutdown()
     /* rank 0 is responsible for writing the compressed darshan job information */
     if(my_rank == 0)
     {
-        void *pointers[2] = {&out_job, final_core->log_exemnt_p};
+        void *pointers[2] = {final_core->log_job_p, final_core->log_exemnt_p};
         int lengths[2] = {sizeof(struct darshan_job), strlen(final_core->log_exemnt_p)};
         int comp_buf_sz = 0;
 
@@ -564,11 +490,11 @@ void darshan_core_shutdown()
     if(internal_timing_flag)
         rec1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
     /* write the record name->id hash to the log file */
-    out_header.name_map.off = gz_fp;
+    final_core->log_hdr_p->name_map.off = gz_fp;
     ret = darshan_log_write_name_record_hash(log_fh, final_core, &gz_fp);
-    out_header.name_map.len = gz_fp - out_header.name_map.off;
+    final_core->log_hdr_p->name_map.len = gz_fp - final_core->log_hdr_p->name_map.off;
 
-    /* error out if unable to write record hash */
+    /* error out if unable to write the name record hash */
     DARSHAN_MPI_CALL(PMPI_Allreduce)(&ret, &all_ret, 1, MPI_INT,
         MPI_LOR, MPI_COMM_WORLD);
     if(all_ret != 0)
@@ -591,11 +517,10 @@ void darshan_core_shutdown()
     assert(mod_shared_recs);
 
     /* loop over globally used darshan modules and:
-     *      - perform shared file reductions, if possible
      *      - get final output buffer
      *      - compress (zlib) provided output buffer
      *      - append compressed buffer to log file
-     *      - add module index info (file offset/length) to log header
+     *      - add module map info (file offset/length) to log header
      *      - shutdown the module
      */
     for(i = 0; i < DARSHAN_MAX_MODS; i++)
@@ -611,8 +536,8 @@ void darshan_core_shutdown()
         {
             if(my_rank == 0)
             {
-                out_header.mod_map[i].off = 0;
-                out_header.mod_map[i].len = 0;
+                final_core->log_hdr_p->mod_map[i].off = 0;
+                final_core->log_hdr_p->mod_map[i].len = 0;
             }
             continue;
         }
@@ -620,8 +545,7 @@ void darshan_core_shutdown()
         if(internal_timing_flag)
             mod1[i] = DARSHAN_MPI_CALL(PMPI_Wtime)();
 
-        /* set the shared file list for this module */
-        memset(mod_shared_recs, 0, shared_rec_cnt * sizeof(darshan_record_id));
+        /* set the shared record list for this module */
         for(j = 0; j < shared_rec_cnt; j++)
         {
             HASH_FIND(hlink, final_core->name_hash, &shared_recs[j],
@@ -636,7 +560,7 @@ void darshan_core_shutdown()
         /* if module is registered locally, get the corresponding output buffer
          * 
          * NOTE: this function can be used to run collective operations across
-         * modules, if there are file records shared globally.
+         * modules, if there are records shared globally.
          */
         if(this_mod)
         {
@@ -647,9 +571,10 @@ void darshan_core_shutdown()
         }
 
         /* append this module's data to the darshan log */
-        out_header.mod_map[i].off = gz_fp;
+        final_core->log_hdr_p->mod_map[i].off = gz_fp;
         ret = darshan_log_append_all(log_fh, final_core, mod_buf, mod_buf_sz, &gz_fp);
-        out_header.mod_map[i].len = gz_fp - out_header.mod_map[i].off;
+        final_core->log_hdr_p->mod_map[i].len =
+            gz_fp - final_core->log_hdr_p->mod_map[i].off;
 
         /* error out if the log append failed */
         DARSHAN_MPI_CALL(PMPI_Allreduce)(&ret, &all_ret, 1, MPI_INT,
@@ -679,26 +604,25 @@ void darshan_core_shutdown()
 
     if(internal_timing_flag)
         header1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
-    /* write out log header, after running 2 reduction on header variables:
+    /* write out log header, after running 2 reductions on header variables:
      *  1) reduce 'partial_flag' variable to determine which modules ran out
-     *     of memory for storing I/O data
+     *     of memory for storing data
      *  2) reduce 'mod_ver' array to determine which log format version each
      *     module used for this output log
      */
-    DARSHAN_MPI_CALL(PMPI_Reduce)(&(final_core->log_hdr_p->partial_flag),
-        &(out_header.partial_flag), 1, MPI_UINT32_T, MPI_BOR, 0, MPI_COMM_WORLD);
-    DARSHAN_MPI_CALL(PMPI_Reduce)(&(final_core->log_hdr_p->mod_ver),
-        &(out_header.mod_ver), DARSHAN_MAX_MODS, MPI_UINT32_T, MPI_MAX,
-        0, MPI_COMM_WORLD);
     if(my_rank == 0)
     {
         /* rank 0 is responsible for writing the log header */
-        /* initialize the remaining header fields */
-        strcpy(out_header.version_string, DARSHAN_LOG_VERSION);
-        out_header.magic_nr = DARSHAN_MAGIC_NR;
-        out_header.comp_type = DARSHAN_ZLIB_COMP;
+        final_core->log_hdr_p->comp_type = DARSHAN_ZLIB_COMP;
+
+        DARSHAN_MPI_CALL(PMPI_Reduce)(
+            MPI_IN_PLACE, &(final_core->log_hdr_p->partial_flag),
+            1, MPI_UINT32_T, MPI_BOR, 0, MPI_COMM_WORLD);
+        DARSHAN_MPI_CALL(PMPI_Reduce)(
+            MPI_IN_PLACE, &(final_core->log_hdr_p->mod_ver),
+            DARSHAN_MAX_MODS, MPI_UINT32_T, MPI_MAX, 0, MPI_COMM_WORLD);
 
-        all_ret = DARSHAN_MPI_CALL(PMPI_File_write_at)(log_fh, 0, &out_header,
+        all_ret = DARSHAN_MPI_CALL(PMPI_File_write_at)(log_fh, 0, final_core->log_hdr_p,
             sizeof(struct darshan_header), MPI_BYTE, &status);
         if(all_ret != MPI_SUCCESS)
         {
@@ -707,6 +631,15 @@ void darshan_core_shutdown()
             unlink(logfile_name);
         }
     }
+    else
+    {
+        DARSHAN_MPI_CALL(PMPI_Reduce)(
+            &(final_core->log_hdr_p->partial_flag), &(final_core->log_hdr_p->partial_flag),
+            1, MPI_UINT32_T, MPI_BOR, 0, MPI_COMM_WORLD);
+        DARSHAN_MPI_CALL(PMPI_Reduce)(
+            &(final_core->log_hdr_p->mod_ver), &(final_core->log_hdr_p->mod_ver),
+            DARSHAN_MAX_MODS, MPI_UINT32_T, MPI_MAX, 0, MPI_COMM_WORLD);
+    }
 
     /* error out if unable to write log header */
     DARSHAN_MPI_CALL(PMPI_Bcast)(&all_ret, 1, MPI_INT, 0, MPI_COMM_WORLD);
@@ -758,11 +691,6 @@ void darshan_core_shutdown()
         }
     }
 
-#ifdef __DARSHAN_ENABLE_MMAP_LOGS
-    /* remove the temporary mmap log files */
-    unlink(final_core->mmap_log_name);
-#endif
-
     free(logfile_name);
     free(shared_recs);
     free(mod_shared_recs);
@@ -824,6 +752,80 @@ void darshan_core_shutdown()
 
 /* *********************************** */
 
+static void *darshan_init_mmap_log(struct darshan_core_runtime* core, int jobid)
+{
+    int ret;
+    int mmap_fd;
+    int mmap_size;
+    int sys_page_size;
+    char cuser[L_cuserid] = {0};
+    char *envstr;
+    char *mmap_log_path;
+    void *mmap_p;
+
+    sys_page_size = sysconf(_SC_PAGESIZE);
+    assert(sys_page_size > 0);
+
+    mmap_size = sizeof(struct darshan_header) + DARSHAN_JOB_RECORD_SIZE +
+        + DARSHAN_NAME_RECORD_BUF_SIZE + darshan_mod_mem_quota;
+    if(mmap_size % sys_page_size)
+        mmap_size = ((mmap_size / sys_page_size) + 1) * sys_page_size;
+
+    envstr = getenv(DARSHAN_MMAP_LOG_PATH_OVERRIDE);
+    if(envstr)
+        mmap_log_path = envstr;
+    else
+        mmap_log_path = DARSHAN_DEF_MMAP_LOG_PATH;
+
+    darshan_get_user_name(cuser);
+
+    /* construct a unique temporary log file name for this process
+     * to write mmap log data to
+     */
+    snprintf(core->mmap_log_name, PATH_MAX,
+        "/%s/%s_%s_id%d_mmap-log-%d.darshan",
+        mmap_log_path, cuser, __progname, jobid, my_rank);
+
+    /* create the temporary mmapped darshan log */
+    mmap_fd = open(core->mmap_log_name, O_CREAT|O_RDWR|O_EXCL , 0644);
+    if(mmap_fd < 0)
+    {
+        fprintf(stderr, "darshan library warning: "
+            "unable to create darshan log file %s\n", core->mmap_log_name);
+        return(NULL);
+    }
+
+    /* TODO: ftruncate or just zero fill? */
+    /* allocate the necessary space in the log file */
+    ret = ftruncate(mmap_fd, mmap_size);
+    if(ret < 0)
+    {
+        fprintf(stderr, "darshan library warning: "
+            "unable to allocate darshan log file %s\n", core->mmap_log_name);
+        close(mmap_fd);
+        unlink(core->mmap_log_name);
+        return(NULL);
+    }
+
+    /* create the memory map for darshan's data structures so they are
+     * persisted to file as the application executes
+     */
+    mmap_p = mmap(NULL, mmap_size, PROT_WRITE, MAP_SHARED, mmap_fd, 0);
+    if(mmap_p == MAP_FAILED)
+    {
+        fprintf(stderr, "darshan library warning: "
+            "unable to mmap darshan log file %s\n", core->mmap_log_name);
+        close(mmap_fd);
+        unlink(core->mmap_log_name);
+        return(NULL);
+    }
+
+    /* close darshan log file (this does *not* unmap the log file) */
+    close(mmap_fd);
+
+    return(mmap_p);
+}
+
 /* record any hints used to write the darshan log in the job data */
 static void darshan_log_record_hints_and_ver(struct darshan_core_runtime* core)
 {
@@ -884,7 +886,7 @@ static int mnt_data_cmp(const void* a, const void* b)
 }
 
 /* adds an entry to table of mounted file systems */
-static void add_entry(char* buf, int* space_left, struct mntent *entry)
+static void add_entry(char* buf, int* space_left, struct mntent* entry)
 {
     int ret;
     char tmp_mnt[256];
@@ -1221,38 +1223,37 @@ static void darshan_get_logfile_name(char* logfile_name, int jobid, struct tm* s
     return;
 }
 
-static void darshan_add_name_record_ref(struct darshan_core_runtime *core,
+static int darshan_add_name_record_ref(struct darshan_core_runtime *core,
     darshan_record_id rec_id, const char *name, darshan_module_id mod_id)
 {
     struct darshan_core_name_record_ref *ref;
     int record_size = sizeof(darshan_record_id) + strlen(name) + 1;
 
-    if((record_size + core->log_hdr_p->name_map.len) > DARSHAN_NAME_RECORD_BUF_SIZE)
-        return;
+    if((record_size + core->name_mem_used) > DARSHAN_NAME_RECORD_BUF_SIZE)
+        return(0);
 
     ref = malloc(sizeof(*ref));
-    if(ref)
-    {
-        memset(ref, 0, sizeof(*ref));
-
-        ref->name_record = (struct darshan_name_record *)
-            ((char *)core->log_name_p + core->log_hdr_p->name_map.len);
-        memset(ref->name_record, 0, record_size);
-        ref->name_record->id = rec_id;
-        strcpy(ref->name_record->name, name);
-
-        if(!DARSHAN_MOD_FLAG_ISSET(ref->mod_flags, mod_id))
-        {
-            DARSHAN_MOD_FLAG_SET(ref->mod_flags, mod_id);
-        }
-
-        /* add the record to the hash table */
-        HASH_ADD(hlink, core->name_hash, name_record->id,
-            sizeof(darshan_record_id), ref);
-        core->log_hdr_p->name_map.len += record_size;
-    }
+    if(!ref)
+        return(0);
+    memset(ref, 0, sizeof(*ref));
+
+    /* initialize the name record */
+    ref->name_record = (struct darshan_name_record *)
+        ((char *)core->log_name_p + core->name_mem_used);
+    memset(ref->name_record, 0, record_size);
+    ref->name_record->id = rec_id;
+    strcpy(ref->name_record->name, name);
+    DARSHAN_MOD_FLAG_SET(ref->mod_flags, mod_id);
+
+    /* add the record to the hash table */
+    HASH_ADD(hlink, core->name_hash, name_record->id,
+        sizeof(darshan_record_id), ref);
+    core->name_mem_used += record_size;
+#ifdef __DARSHAN_ENABLE_MMAP_LOGS
+    core->log_hdr_p->name_map.len += record_size;
+#endif
 
-    return;
+    return(1);
 }
 
 static void darshan_get_shared_records(struct darshan_core_runtime *core,
@@ -1304,7 +1305,7 @@ static void darshan_get_shared_records(struct darshan_core_runtime *core,
         }
     }
 
-    /* now allreduce so everyone agrees which files are shared and
+    /* now allreduce so everyone agrees which records are shared and
      * which modules accessed them collectively
      */
     DARSHAN_MPI_CALL(PMPI_Allreduce)(mod_flags, global_mod_flags, tmp_cnt,
@@ -1319,7 +1320,7 @@ static void darshan_get_shared_records(struct darshan_core_runtime *core,
 
             /* set global_mod_flags so we know which modules collectively
              * accessed this module. we need this info to support shared
-             * file reductions
+             * record reductions
              */
             HASH_FIND(hlink, core->name_hash, &id_array[i], sizeof(darshan_record_id), ref);
             assert(ref);
@@ -1496,7 +1497,8 @@ static int darshan_log_write_name_record_hash(MPI_File log_fh,
     int name_rec_buf_len;
     int ret;
 
-    name_rec_buf_len = core->log_hdr_p->name_map.len;
+    /* remove globally shared name records from non-zero ranks */
+    name_rec_buf_len = core->name_mem_used;
     if(my_rank > 0)
     {
         name_rec = core->log_name_p;
@@ -1512,6 +1514,9 @@ static int darshan_log_write_name_record_hash(MPI_File log_fh,
 
             if(ref->global_mod_flags)
             {
+                /* this record is shared globally, move to the temporary
+                 * shared record buffer and update hash references
+                 */
                 HASH_DELETE(hlink, core->name_hash, ref);
                 memcpy(shared_buf, name_rec, rec_len);
                 ref->name_record = (struct darshan_name_record *)shared_buf;
@@ -1523,6 +1528,9 @@ static int darshan_log_write_name_record_hash(MPI_File log_fh,
             }
             else
             {
+                /* this record is not shared, but we still may need to
+                 * move it forward in our buffer and update hash references
+                 */
                 if(my_buf != (char *)name_rec)
                 {
                     HASH_DELETE(hlink, core->name_hash, ref);
@@ -1538,8 +1546,12 @@ static int darshan_log_write_name_record_hash(MPI_File log_fh,
             name_rec = (struct darshan_name_record *)tmp_p;
             name_rec_buf_len -= rec_len;
         }
-        name_rec_buf_len = core->log_hdr_p->name_map.len - shared_buf_len;
+        name_rec_buf_len = core->name_mem_used - shared_buf_len;
 
+        /* append the shared records back to the end of the name record
+         * buffer and update hash table references so we can still
+         * reference these records as modules shutdown
+         */
         name_rec = (struct darshan_name_record *)core->comp_buf;
         while(shared_buf_len > 0)
         {
@@ -1678,7 +1690,6 @@ static void darshan_core_cleanup(struct darshan_core_runtime* core)
 
 /* ********************************************************* */
 
-/* TODO: do we alloc new space as we go or just do everything up front? */
 void darshan_core_register_module(
     darshan_module_id mod_id,
     struct darshan_module_funcs *funcs,
@@ -1712,7 +1723,7 @@ void darshan_core_register_module(
     memset(mod, 0, sizeof(*mod));
 
     /* set module's record buffer and max memory usage */
-    mod_mem_avail = DARSHAN_MOD_MEM_MAX - darshan_core->mod_mem_used;
+    mod_mem_avail = darshan_mod_mem_quota - darshan_core->mod_mem_used;
     if(mod_mem_avail >= mod_mem_req)
         mod->rec_mem_avail = mod_mem_req;
     else
@@ -1724,12 +1735,11 @@ void darshan_core_register_module(
     /* register module with darshan */
     darshan_core->mod_array[mod_id] = mod;
     darshan_core->mod_mem_used += mod->rec_mem_avail;
-
-    /* update darshan header */
-    /* TODO: ifdef wrap for mmap ? */
     darshan_core->log_hdr_p->mod_ver[mod_id] = darshan_module_versions[mod_id];
+#ifdef __DARSHAN_ENABLE_MMAP_LOGS
     darshan_core->log_hdr_p->mod_map[mod_id].off =
         ((char *)mod->rec_buf_start - (char *)darshan_core->log_hdr_p);
+#endif
 
     *inout_mod_buf_size = mod->rec_mem_avail;
     DARSHAN_CORE_UNLOCK();
@@ -1743,35 +1753,26 @@ void darshan_core_register_module(
     return;
 }
 
-/* TODO: */
+/* NOTE: we currently don't really have a simple way of returning the
+ * memory allocated to this module back to darshan to hand out to
+ * other modules, so all we do is disable the module so darshan does
+ * not attempt to call into it at shutdown time
+ */
 void darshan_core_unregister_module(
     darshan_module_id mod_id)
 {
-    struct darshan_core_name_record_ref *ref, *tmp;
-
     if(!darshan_core)
         return;
 
     DARSHAN_CORE_LOCK();
-
-    /* iterate all records and disassociate this module from them */
-    HASH_ITER(hlink, darshan_core->name_hash, ref, tmp)
-    {
-        /* disassociate this module from the given record id */
-        DARSHAN_MOD_FLAG_UNSET(ref->mod_flags, mod_id);
-        if(!(ref->mod_flags))
-        {
-            /* if no other modules are associated with this rec, delete it */
-            HASH_DELETE(hlink, darshan_core->name_hash, ref);
-        }
-    }
-
     /* update darshan internal structures and header */
     free(darshan_core->mod_array[mod_id]);
     darshan_core->mod_array[mod_id] = NULL;
+    darshan_core->log_hdr_p->mod_ver[mod_id] = 0;
+#ifdef __DARSHAN_ENABLE_MMAP_LOGS
     darshan_core->log_hdr_p->mod_map[mod_id].off =
         darshan_core->log_hdr_p->mod_map[mod_id].len = 0;
-
+#endif
     DARSHAN_CORE_UNLOCK();
 
     return;
@@ -1793,6 +1794,7 @@ void *darshan_core_register_record(
 {
     struct darshan_core_name_record_ref *ref;
     void *rec_buf;
+    int ret;
 
     if(!darshan_core)
         return(NULL);
@@ -1817,14 +1819,26 @@ void *darshan_core_register_record(
             sizeof(darshan_record_id), ref);
         if(!ref)
         {
-            darshan_add_name_record_ref(darshan_core, rec_id, name, mod_id);
+            ret = darshan_add_name_record_ref(darshan_core, rec_id, name, mod_id);
+            if(ret == 0)
+            {
+                DARSHAN_MOD_FLAG_SET(darshan_core->log_hdr_p->partial_flag, mod_id);
+                DARSHAN_CORE_UNLOCK();
+                return(NULL);
+            }
+        }
+        else
+        {
+            DARSHAN_MOD_FLAG_SET(ref->mod_flags, mod_id);
         }
     }
 
     rec_buf = darshan_core->mod_array[mod_id]->rec_buf_p;
     darshan_core->mod_array[mod_id]->rec_buf_p += rec_len;
     darshan_core->mod_array[mod_id]->rec_mem_avail -= rec_len;
-    darshan_core->log_hdr_p->mod_map[mod_id].len += rec_len; /* XXX */
+#ifdef __DARSHAN_ENABLE_MMAP_LOGS
+    darshan_core->log_hdr_p->mod_map[mod_id].len += rec_len;
+#endif
     DARSHAN_CORE_UNLOCK();
 
     if(file_alignment)
@@ -1833,13 +1847,6 @@ void *darshan_core_register_record(
     return(rec_buf);;
 }
 
-/* TODO: */
-void darshan_core_unregister_record(
-    darshan_record_id rec_id,
-    darshan_module_id mod_id)
-{
-}
-
 double darshan_core_wtime()
 {
     if(!darshan_core)



View it on GitLab: https://xgitlab.cels.anl.gov/darshan/darshan/commit/d8f3575ac292132eea109c725114bc27883d012d
-------------- next part --------------
An HTML attachment was scrubbed...
URL: <http://lists.mcs.anl.gov/pipermail/darshan-commits/attachments/20160608/8d5facdc/attachment-0001.html>


More information about the Darshan-commits mailing list