[Darshan-commits] [Git][darshan/darshan][mmap-dev] 4 commits: update utilities and log format to use base rec

Shane Snyder xgitlab at cels.anl.gov
Thu Dec 17 16:55:32 CST 2015


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


Commits:
be3912a8 by Shane Snyder at 2015-12-09T10:28:40Z
update utilities and log format to use base rec

- - - - -
fe340438 by Shane Snyder at 2015-12-14T12:48:15Z
add an aggregation routine to module logutils

- - - - -
a84e8657 by Shane Snyder at 2015-12-17T16:54:46Z
integrate normal darshan shutdown back into code

- - - - -
80c07c27 by Shane Snyder at 2015-12-17T16:55:11Z
more darshan-util updates for handling tmp logs

- - - - -


18 changed files:

- darshan-bgq-log-format.h
- darshan-hdf5-log-format.h
- darshan-mpiio-log-format.h
- darshan-null-log-format.h
- darshan-pnetcdf-log-format.h
- darshan-runtime/lib/darshan-core.c
- darshan-util/Makefile.in
- darshan-util/darshan-analyzer.c
- darshan-util/darshan-bgq-logutils.c
- darshan-util/darshan-convert.c
- darshan-util/darshan-hdf5-logutils.c
- darshan-util/darshan-logutils.h
- darshan-util/darshan-mpiio-logutils.c
- darshan-util/darshan-null-logutils.c
- darshan-util/darshan-parser.c
- darshan-util/darshan-pnetcdf-logutils.c
- darshan-util/darshan-posix-logutils.c
- darshan-util/darshan-stitch-logs.c


Changes:

=====================================
darshan-bgq-log-format.h
=====================================
--- a/darshan-bgq-log-format.h
+++ b/darshan-bgq-log-format.h
@@ -53,8 +53,7 @@ enum darshan_bgq_f_indices
  */
 struct darshan_bgq_record
 {
-    darshan_record_id f_id;
-    int64_t rank;
+    struct darshan_base_record base_rec;
     int alignment;
     int64_t counters[BGQ_NUM_INDICES];
     double fcounters[BGQ_F_NUM_INDICES];


=====================================
darshan-hdf5-log-format.h
=====================================
--- a/darshan-hdf5-log-format.h
+++ b/darshan-hdf5-log-format.h
@@ -47,8 +47,7 @@ enum darshan_hdf5_f_indices
  */
 struct darshan_hdf5_file
 {
-    darshan_record_id f_id;
-    int64_t rank;
+    struct darshan_base_record base_rec;
     int64_t counters[HDF5_NUM_INDICES];
     double fcounters[HDF5_F_NUM_INDICES];
 };


=====================================
darshan-mpiio-log-format.h
=====================================
--- a/darshan-mpiio-log-format.h
+++ b/darshan-mpiio-log-format.h
@@ -146,8 +146,7 @@ enum darshan_mpiio_f_indices
  */
 struct darshan_mpiio_file
 {
-    darshan_record_id f_id;
-    int64_t rank;
+    struct darshan_base_record base_rec;
     int64_t counters[MPIIO_NUM_INDICES];
     double fcounters[MPIIO_F_NUM_INDICES];
 };


=====================================
darshan-null-log-format.h
=====================================
--- a/darshan-null-log-format.h
+++ b/darshan-null-log-format.h
@@ -50,8 +50,7 @@ enum darshan_null_f_indices
  */
 struct darshan_null_record
 {
-    darshan_record_id f_id;
-    int64_t rank;
+    struct darshan_base_record base_rec;
     int64_t counters[NULL_NUM_INDICES];
     double fcounters[NULL_F_NUM_INDICES];
 };


=====================================
darshan-pnetcdf-log-format.h
=====================================
--- a/darshan-pnetcdf-log-format.h
+++ b/darshan-pnetcdf-log-format.h
@@ -49,8 +49,7 @@ enum darshan_pnetcdf_f_indices
  */
 struct darshan_pnetcdf_file
 {
-    darshan_record_id f_id;
-    int64_t rank;
+    struct darshan_base_record base_rec;
     int64_t counters[PNETCDF_NUM_INDICES];
     double fcounters[PNETCDF_F_NUM_INDICES];
 };


=====================================
darshan-runtime/lib/darshan-core.c
=====================================
--- a/darshan-runtime/lib/darshan-core.c
+++ b/darshan-runtime/lib/darshan-core.c
@@ -93,6 +93,9 @@ 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_record_hashref(
+    struct darshan_core_runtime *core, char *name,
+    darshan_record_id id, struct darshan_core_record_ref **ref);
 static void darshan_block_size_from_path(
     const char *path, int *block_size);
 static void darshan_get_logfile_name(
@@ -163,7 +166,7 @@ void darshan_core_initialize(int argc, char **argv)
         }
 
         /* avoid floating point errors on faulty input */
-        if (darshan_mem_alignment < 1)
+        if(darshan_mem_alignment < 1)
         {
             darshan_mem_alignment = 1;
         }
@@ -312,11 +315,34 @@ void darshan_core_initialize(int argc, char **argv)
 
 void darshan_core_shutdown()
 {
-    int i;
     struct darshan_core_runtime *final_core;
     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;
+    double open1, open2;
+    double job1, job2;
+    double rec1, rec2;
+    double mod1[DARSHAN_MAX_MODS] = {0};
+    double mod2[DARSHAN_MAX_MODS] = {0};
+    double header1, header2;
+    char *logfile_name;
+    int local_mod_use[DARSHAN_MAX_MODS] = {0};
+    int global_mod_use_count[DARSHAN_MAX_MODS] = {0};
+    darshan_record_id *shared_recs;
+    darshan_record_id *mod_shared_recs;
+    int shared_rec_cnt = 0;
+    int ret = 0;
+    int all_ret = 0;
+    int i;
+    uint64_t gz_fp = 0;
+    struct darshan_header out_header;
+    struct darshan_job out_job;
+    MPI_File log_fh;
+    MPI_Status status;
 
     if(getenv("DARSHAN_INTERNAL_TIMING"))
         internal_timing_flag = 1;
@@ -333,34 +359,412 @@ void darshan_core_shutdown()
     }
     final_core = darshan_core;
     darshan_core = NULL;
+    DARSHAN_CORE_UNLOCK();
+
+    memcpy(&out_job, final_core->log_job_p, sizeof(struct darshan_job));
+
+    /* XXX just copy mmap files somewhere else to avoid corruption */
+    DARSHAN_MPI_CALL(PMPI_Barrier)(MPI_COMM_WORLD);
+    if(my_rank == 0)
+        system("cp /tmp/darshan* ~/Desktop");
+    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.
+     */
+    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;
+    snprintf(m, meta_remain, "darshan_shutdown=yes\n");
 
     for(i = 0; i < DARSHAN_MAX_MODS; i++)
     {
         if(final_core->mod_array[i])
         {
+            local_mod_use[i] = 1;
             final_core->mod_array[i]->mod_funcs.begin_shutdown();
         }
     }
-    DARSHAN_CORE_UNLOCK();
+
+    logfile_name = malloc(PATH_MAX);
+    if(!logfile_name)
+    {
+        darshan_core_cleanup(final_core);
+        return;
+    }
+
+    /* set the log file name on rank 0 */
+    if(my_rank == 0)
+    {
+        /* use human readable start time format in log filename */
+        start_time_tmp = final_core->log_job_p->start_time;
+        start_tm = localtime(&start_time_tmp);
+
+        darshan_get_logfile_name(logfile_name, final_core->log_job_p->jobid, start_tm);
+    }
+
+    /* broadcast log file name */
+    DARSHAN_MPI_CALL(PMPI_Bcast)(logfile_name, PATH_MAX, MPI_CHAR, 0,
+        MPI_COMM_WORLD);
+
+    if(strlen(logfile_name) == 0)
+    {
+        /* failed to generate log file name */
+        free(logfile_name);
+        darshan_core_cleanup(final_core);
+        return;
+    }
 
     final_core->log_job_p->end_time = time(NULL);
 
+    /* reduce to report first start time 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_LONG_LONG, MPI_MIN, 0, MPI_COMM_WORLD);
+    DARSHAN_MPI_CALL(PMPI_Reduce)(&final_core->log_job_p->end_time, &last_end_time,
+        1, MPI_LONG_LONG, MPI_MAX, 0, MPI_COMM_WORLD);
+    if(my_rank == 0)
+    {
+        out_job.start_time = first_start_time;
+        out_job.end_time = last_end_time;
+    }
+
+    /* reduce the number of times a module was opened globally and bcast to everyone */
+    DARSHAN_MPI_CALL(PMPI_Allreduce)(local_mod_use, global_mod_use_count,
+        DARSHAN_MAX_MODS, MPI_INT, MPI_SUM, MPI_COMM_WORLD);
+
+    /* get a list of records which are shared across all processes */
+    darshan_get_shared_records(final_core, &shared_recs, &shared_rec_cnt);
+
+    if(internal_timing_flag)
+        open1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+    /* collectively open the darshan log file */
+    ret = darshan_log_open_all(logfile_name, &log_fh);
+    if(internal_timing_flag)
+        open2 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+
+    /* error out if unable to open log file */
+    DARSHAN_MPI_CALL(PMPI_Allreduce)(&ret, &all_ret, 1, MPI_INT,
+        MPI_LOR, MPI_COMM_WORLD);
+    if(all_ret != 0)
+    {
+        if(my_rank == 0)
+        {
+            fprintf(stderr, "darshan library warning: unable to open log file %s\n",
+                logfile_name);
+        }
+        free(logfile_name);
+        darshan_core_cleanup(final_core);
+        return;
+    }
+
+    if(internal_timing_flag)
+        job1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+    /* 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};
+        int lengths[2] = {sizeof(struct darshan_job), strlen(final_core->log_exemnt_p)};
+        int comp_buf_sz = 0;
+
+        /* compress the job info and the trailing mount/exe data */
+        all_ret = darshan_deflate_buffer(pointers, lengths, 2,
+            final_core->comp_buf, &comp_buf_sz);
+        if(all_ret)
+        {
+            fprintf(stderr, "darshan library warning: unable to compress job data\n");
+            unlink(logfile_name);
+        }
+        else
+        {
+            /* write the job information, preallocing space for the log header */
+            gz_fp += sizeof(struct darshan_header);
+            all_ret = DARSHAN_MPI_CALL(PMPI_File_write_at)(log_fh, gz_fp,
+                final_core->comp_buf, comp_buf_sz, MPI_BYTE, &status);
+            if(all_ret != MPI_SUCCESS)
+            {
+                fprintf(stderr,
+                        "darshan library warning: unable to write job data to log file %s\n",
+                        logfile_name);
+                unlink(logfile_name);
+
+            }
+            gz_fp += comp_buf_sz;
+        }
+    }
+
+    /* error out if unable to write job information */
+    DARSHAN_MPI_CALL(PMPI_Bcast)(&all_ret, 1, MPI_INT, 0, MPI_COMM_WORLD);
+    if(all_ret != 0)
+    {
+        free(logfile_name);
+        darshan_core_cleanup(final_core);
+        return;
+    }
+    if(internal_timing_flag)
+        job2 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+
+    /* error out if unable to write job information */
+    DARSHAN_MPI_CALL(PMPI_Bcast)(&all_ret, 1, MPI_INT, 0, MPI_COMM_WORLD);
+    if(all_ret != 0)
+    {
+        free(logfile_name);
+        darshan_core_cleanup(final_core);
+        return;
+    }
+    if(internal_timing_flag)
+        job2 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+
+    if(internal_timing_flag)
+        rec1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+    /* write the record name->id hash to the log file */
+    out_header.rec_map.off = gz_fp;
+    ret = darshan_log_write_record_hash(log_fh, final_core, &gz_fp);
+    out_header.rec_map.len = gz_fp - out_header.rec_map.off;
+
+    /* error out if unable to write record hash */
+    DARSHAN_MPI_CALL(PMPI_Allreduce)(&ret, &all_ret, 1, MPI_INT,
+        MPI_LOR, MPI_COMM_WORLD);
+    if(all_ret != 0)
+    {
+        if(my_rank == 0)
+        {
+            fprintf(stderr,
+                "darshan library warning: unable to write record hash to log file %s\n",
+                logfile_name);
+            unlink(logfile_name);
+        }
+        free(logfile_name);
+        darshan_core_cleanup(final_core);
+        return;
+    }
+    if(internal_timing_flag)
+        rec2 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+
+    mod_shared_recs = malloc(shared_rec_cnt * sizeof(darshan_record_id));
+    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
+     *      - shutdown the module
+     */
+    for(i = 0; i < DARSHAN_MAX_MODS; i++)
+    {
+        struct darshan_core_module* this_mod = final_core->mod_array[i];
+        struct darshan_core_record_ref *ref = NULL;
+        int mod_shared_rec_cnt = 0;
+        void* mod_buf = NULL;
+        int mod_buf_sz = 0;
+        int j;
+
+        if(global_mod_use_count[i] == 0)
+        {
+            if(my_rank == 0)
+            {
+                out_header.mod_map[i].off = 0;
+                out_header.mod_map[i].len = 0;
+            }
+            continue;
+        }
+
+        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));
+        for(j = 0; j < shared_rec_cnt; j++)
+        {
+            HASH_FIND(hlink, final_core->rec_hash, &shared_recs[j],
+                sizeof(darshan_record_id), ref);
+            assert(ref);
+            if(DARSHAN_MOD_FLAG_ISSET(ref->global_mod_flags, i))
+            {
+                mod_shared_recs[mod_shared_rec_cnt++] = shared_recs[j];
+            }
+        }
+
+        /* 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.
+         */
+        if(this_mod)
+        {
+            this_mod->mod_funcs.get_output_data(MPI_COMM_WORLD, mod_shared_recs,
+                mod_shared_rec_cnt, &mod_buf, &mod_buf_sz);
+        }
+
+        /* append this module's data to the darshan log */
+        out_header.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;
+
+        /* error out if the log append failed */
+        DARSHAN_MPI_CALL(PMPI_Allreduce)(&ret, &all_ret, 1, MPI_INT,
+            MPI_LOR, MPI_COMM_WORLD);
+        if(all_ret != 0)
+        {
+            if(my_rank == 0)
+            {
+                fprintf(stderr,
+                    "darshan library warning: unable to write %s module data to log file %s\n",
+                    darshan_module_names[i], logfile_name);
+                unlink(logfile_name);
+            }
+            free(logfile_name);
+            darshan_core_cleanup(final_core);
+            return;
+        }
+
+        /* shutdown module if registered locally */
+        if(this_mod)
+        {
+            this_mod->mod_funcs.shutdown();
+        }
+        if(internal_timing_flag)
+            mod2[i] = DARSHAN_MPI_CALL(PMPI_Wtime)();
+    }
+
+    if(internal_timing_flag)
+        header1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+    /* write out log header, after running 2 reduction on header variables:
+     *  1) reduce 'partial_flag' variable to determine which modules ran out
+     *     of memory for storing I/O 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);
+    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;
+
+        all_ret = DARSHAN_MPI_CALL(PMPI_File_write_at)(log_fh, 0, &out_header,
+            sizeof(struct darshan_header), MPI_BYTE, &status);
+        if(all_ret != MPI_SUCCESS)
+        {
+            fprintf(stderr, "darshan library warning: unable to write header to log file %s\n",
+                    logfile_name);
+            unlink(logfile_name);
+        }
+    }
+
+    /* error out if unable to write log header */
+    DARSHAN_MPI_CALL(PMPI_Bcast)(&all_ret, 1, MPI_INT, 0, MPI_COMM_WORLD);
+    if(all_ret != 0)
+    {
+        free(logfile_name);
+        darshan_core_cleanup(final_core);
+        return;
+    }
+    if(internal_timing_flag)
+        header2 = DARSHAN_MPI_CALL(PMPI_Wtime)();
+
+    DARSHAN_MPI_CALL(PMPI_File_close)(&log_fh);
+
+    /* if we got this far, there are no errors, so rename from *.darshan_partial
+     * to *-<logwritetime>.darshan, which indicates that this log file is
+     * complete and ready for analysis
+     */
+    if(my_rank == 0)
+    {
+        if(getenv("DARSHAN_LOGFILE"))
+        {
+#ifdef __DARSHAN_GROUP_READABLE_LOGS
+            chmod(logfile_name, (S_IRUSR|S_IRGRP));
+#else
+            chmod(logfile_name, (S_IRUSR));
+#endif
+        }
+        else
+        {
+            char* tmp_index;
+            double end_log_time;
+            char* new_logfile_name;
+
+            new_logfile_name = malloc(PATH_MAX);
+            if(new_logfile_name)
+            {
+                new_logfile_name[0] = '\0';
+                end_log_time = DARSHAN_MPI_CALL(PMPI_Wtime)();
+                strcat(new_logfile_name, logfile_name);
+                tmp_index = strstr(new_logfile_name, ".darshan_partial");
+                sprintf(tmp_index, "_%d.darshan", (int)(end_log_time-start_log_time+1));
+                rename(logfile_name, new_logfile_name);
+                /* set permissions on log file */
+#ifdef __DARSHAN_GROUP_READABLE_LOGS
+                chmod(new_logfile_name, (S_IRUSR|S_IRGRP));
+#else
+                chmod(new_logfile_name, (S_IRUSR));
+#endif
+                free(new_logfile_name);
+            }
+        }
+    }
+
+    /* TODO: remove temporary log files after successfully creating darshan log */
+
+    free(logfile_name);
     darshan_core_cleanup(final_core);
 
     if(internal_timing_flag)
     {
+        double open_tm, open_slowest;
+        double header_tm, header_slowest;
+        double job_tm, job_slowest;
+        double rec_tm, rec_slowest;
+        double mod_tm[DARSHAN_MAX_MODS], mod_slowest[DARSHAN_MAX_MODS];
         double all_tm, all_slowest;
 
         tm_end = DARSHAN_MPI_CALL(PMPI_Wtime)();
 
+        open_tm = open2 - open1;
+        header_tm = header2 - header1;
+        job_tm = job2 - job1;
+        rec_tm = rec2 - rec1;
         all_tm = tm_end - start_log_time;
+        for(i = 0;i < DARSHAN_MAX_MODS; i++)
+        {
+            mod_tm[i] = mod2[i] - mod1[i];
+        }
 
+        DARSHAN_MPI_CALL(PMPI_Reduce)(&open_tm, &open_slowest, 1,
+            MPI_DOUBLE, MPI_MAX, 0, MPI_COMM_WORLD);
+        DARSHAN_MPI_CALL(PMPI_Reduce)(&header_tm, &header_slowest, 1,
+            MPI_DOUBLE, MPI_MAX, 0, MPI_COMM_WORLD);
+        DARSHAN_MPI_CALL(PMPI_Reduce)(&job_tm, &job_slowest, 1,
+            MPI_DOUBLE, MPI_MAX, 0, MPI_COMM_WORLD);
+        DARSHAN_MPI_CALL(PMPI_Reduce)(&rec_tm, &rec_slowest, 1,
+            MPI_DOUBLE, MPI_MAX, 0, MPI_COMM_WORLD);
         DARSHAN_MPI_CALL(PMPI_Reduce)(&all_tm, &all_slowest, 1,
             MPI_DOUBLE, MPI_MAX, 0, MPI_COMM_WORLD);
+        DARSHAN_MPI_CALL(PMPI_Reduce)(mod_tm, mod_slowest, DARSHAN_MAX_MODS,
+            MPI_DOUBLE, MPI_MAX, 0, MPI_COMM_WORLD);
 
         if(my_rank == 0)
         {
             fprintf(stderr, "#darshan:<op>\t<nprocs>\t<time>\n");
+            fprintf(stderr, "darshan:log_open\t%d\t%f\n", nprocs, open_slowest);
+            fprintf(stderr, "darshan:job_write\t%d\t%f\n", nprocs, job_slowest);
+            fprintf(stderr, "darshan:hash_write\t%d\t%f\n", nprocs, rec_slowest);
+            fprintf(stderr, "darshan:header_write\t%d\t%f\n", nprocs, header_slowest);
+            for(i = 0; i < DARSHAN_MAX_MODS; i++)
+            {
+                if(global_mod_use_count[i])
+                    fprintf(stderr, "darshan:%s_shutdown\t%d\t%f\n", darshan_module_names[i],
+                        nprocs, mod_slowest[i]);
+            }
             fprintf(stderr, "darshan:core_shutdown\t%d\t%f\n", nprocs, all_slowest);
         }
     }
@@ -1035,72 +1439,22 @@ static int darshan_deflate_buffer(void **pointers, int *lengths, int count,
 static int darshan_log_write_record_hash(MPI_File log_fh, struct darshan_core_runtime *core,
     uint64_t *inout_off)
 {
-    int ret;
     struct darshan_core_record_ref *ref, *tmp;
-    uint32_t name_len;
-    size_t record_sz;
-    size_t hash_buf_sz = 0;
-    char *hash_buf;
-    char *hash_buf_off;
-
-    /* allocate a buffer to store at most 64 bytes for each registered record */
-    /* NOTE: this buffer may be reallocated if estimate is too small */
-    hash_buf_sz = core->rec_hash_cnt * 64;
-    hash_buf = malloc(hash_buf_sz);
-    if(!hash_buf)
-    {
-        return(-1);
-    }
+    int ret;
 
     /* serialize the record hash into a buffer for writing */
-    hash_buf_off = hash_buf;
     HASH_ITER(hlink, core->rec_hash, ref, tmp)
     {
         /* to avoid duplicate records, only rank 0 will write shared records */
         if(my_rank > 0 && ref->global_mod_flags)
-            continue;
-
-        name_len = strlen(ref->name);
-        record_sz = sizeof(darshan_record_id) + sizeof(uint32_t) + name_len;
-        /* make sure there is room in the buffer for this record */
-        if((hash_buf_off + record_sz) > (hash_buf + hash_buf_sz))
         {
-            char *tmp_buf;
-            size_t old_buf_sz;
-
-            /* if no room, reallocate the hash buffer at twice the current size */
-            old_buf_sz = hash_buf_off - hash_buf;
-            hash_buf_sz *= 2;
-            tmp_buf = malloc(hash_buf_sz);
-            if(!tmp_buf)
-            {
-                free(hash_buf);
-                return(-1);
-            }
-
-            memcpy(tmp_buf, hash_buf, old_buf_sz);
-            free(hash_buf);
-            hash_buf = tmp_buf;
-            hash_buf_off = hash_buf + old_buf_sz;
+            /* TODO: remove ref */
         }
-
-        /* now serialize the record into the hash buffer.
-         * NOTE: darshan record hash serialization method: 
-         *          ... darshan_record_id | (uint32_t) path_len | path ...
-         */
-        *((darshan_record_id *)hash_buf_off) = ref->id;
-        hash_buf_off += sizeof(darshan_record_id);
-        *((uint32_t *)hash_buf_off) = name_len;
-        hash_buf_off += sizeof(uint32_t);
-        memcpy(hash_buf_off, ref->name, name_len);
-        hash_buf_off += name_len;
     }
-    hash_buf_sz = hash_buf_off - hash_buf;
 
     /* collectively write out the record hash to the darshan log */
-    ret = darshan_log_append_all(log_fh, core, hash_buf, hash_buf_sz, inout_off);
-
-    free(hash_buf);
+    ret = darshan_log_append_all(log_fh, core, core->log_rec_p,
+        core->rec_hash_sz, inout_off);
 
     return(ret);
 }


=====================================
darshan-util/Makefile.in
=====================================
--- a/darshan-util/Makefile.in
+++ b/darshan-util/Makefile.in
@@ -156,7 +156,7 @@ endif
 
 
 clean::
-	rm -f *.o *.po *.a darshan-analyzer darshan-convert darshan-parser jenkins-hash-gen
+	rm -f *.o *.po *.a *.so darshan-analyzer darshan-convert darshan-parser darshan-stitch-logs jenkins-hash-gen
 
 distclean:: clean
 	rm -f darshan-runtime-config.h aclocal.m4 autom4te.cache/* config.status config.log Makefile util/bin/darshan-job-summary.pl


=====================================
darshan-util/darshan-analyzer.c
=====================================
--- a/darshan-util/darshan-analyzer.c
+++ b/darshan-util/darshan-analyzer.c
@@ -38,7 +38,6 @@ int process_log(const char *fname, double *io_ratio, int *used_mpio, int *used_p
     struct darshan_job job;
     struct darshan_mod_logutil_funcs *psx_mod = mod_logutils[DARSHAN_POSIX_MOD];
     struct darshan_posix_file psx_rec;
-    darshan_record_id rec_id;
     int f_count;
     double total_io_time;
     double total_job_time;
@@ -64,7 +63,7 @@ int process_log(const char *fname, double *io_ratio, int *used_mpio, int *used_p
     f_count = 0;
     total_io_time = 0.0;
 
-    while((ret = psx_mod->log_get_record(file, &psx_rec, &rec_id)) == 1)
+    while((ret = psx_mod->log_get_record(file, &psx_rec)) == 1)
     {
         f_count   += 1;
 


=====================================
darshan-util/darshan-bgq-logutils.c
=====================================
--- a/darshan-util/darshan-bgq-logutils.c
+++ b/darshan-util/darshan-bgq-logutils.c
@@ -30,8 +30,7 @@ char *bgq_f_counter_names[] = {
 };
 #undef X
 
-static int darshan_log_get_bgq_rec(darshan_fd fd, void* bgq_buf,
-    darshan_record_id* rec_id);
+static int darshan_log_get_bgq_rec(darshan_fd fd, void* bgq_buf);
 static int darshan_log_put_bgq_rec(darshan_fd fd, void* bgq_buf);
 static void darshan_log_print_bgq_rec(void *file_rec,
     char *file_name, char *mnt_pt, char *fs_type);
@@ -41,10 +40,10 @@ struct darshan_mod_logutil_funcs bgq_logutils =
     .log_get_record = &darshan_log_get_bgq_rec,
     .log_put_record = &darshan_log_put_bgq_rec,
     .log_print_record = &darshan_log_print_bgq_rec,
+    .log_agg_records = NULL /* TODO: how would aggregation work for the BG/Q module ? */
 };
 
-static int darshan_log_get_bgq_rec(darshan_fd fd, void* bgq_buf,
-    darshan_record_id* rec_id)
+static int darshan_log_get_bgq_rec(darshan_fd fd, void* bgq_buf)
 {
     struct darshan_bgq_record *rec;
     int i;
@@ -62,15 +61,14 @@ static int darshan_log_get_bgq_rec(darshan_fd fd, void* bgq_buf,
         if(fd->swap_flag)
         {
             /* swap bytes if necessary */
-            DARSHAN_BSWAP64(&rec->f_id);
-            DARSHAN_BSWAP64(&rec->rank);
+            DARSHAN_BSWAP64(&(rec->base_rec.id));
+            DARSHAN_BSWAP64(&(rec->base_rec.rank));
             for(i=0; i<BGQ_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&rec->counters[i]);
             for(i=0; i<BGQ_F_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&rec->fcounters[i]);
         }
 
-        *rec_id = rec->f_id;
         return(1);
     }
 }
@@ -98,15 +96,17 @@ static void darshan_log_print_bgq_rec(void *file_rec, char *file_name,
     for(i=0; i<BGQ_NUM_INDICES; i++)
     {
         DARSHAN_COUNTER_PRINT(darshan_module_names[DARSHAN_BGQ_MOD],
-            bgq_file_rec->rank, bgq_file_rec->f_id, bgq_counter_names[i],
-            bgq_file_rec->counters[i], file_name, mnt_pt, fs_type);
+            bgq_file_rec->base_rec.rank, bgq_file_rec->base_rec.id,
+            bgq_counter_names[i], bgq_file_rec->counters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     for(i=0; i<BGQ_F_NUM_INDICES; i++)
     {
         DARSHAN_F_COUNTER_PRINT(darshan_module_names[DARSHAN_BGQ_MOD],
-            bgq_file_rec->rank, bgq_file_rec->f_id, bgq_f_counter_names[i],
-            bgq_file_rec->fcounters[i], file_name, mnt_pt, fs_type);
+            bgq_file_rec->base_rec.rank, bgq_file_rec->base_rec.id,
+            bgq_f_counter_names[i], bgq_file_rec->fcounters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     return;


=====================================
darshan-util/darshan-convert.c
=====================================
--- a/darshan-util/darshan-convert.c
+++ b/darshan-util/darshan-convert.c
@@ -342,7 +342,7 @@ int main(int argc, char **argv)
     /* loop over each module and convert it's data to the new format */
     for(i=0; i<DARSHAN_MAX_MODS; i++)
     {
-        darshan_record_id rec_id;
+        struct darshan_base_record *base_rec;
 
         /* check each module for any data */
         if(infile->mod_map[i].len == 0)
@@ -357,7 +357,7 @@ int main(int argc, char **argv)
         /* we have module data to convert */
         memset(mod_buf, 0, DEF_MOD_BUF_SIZE);
 
-        ret = mod_logutils[i]->log_get_record(infile, mod_buf, &rec_id);
+        ret = mod_logutils[i]->log_get_record(infile, mod_buf);
         if(ret != 1)
         {
             fprintf(stderr, "Error: failed to parse the first %s module record.\n",
@@ -371,7 +371,9 @@ int main(int argc, char **argv)
         /* loop over each of the module's records and convert */
         do
         {
-            if(!hash || hash == rec_id)
+            base_rec = (struct darshan_base_record *)mod_buf;
+
+            if(!hash || hash == base_rec->id)
             {
                 ret = mod_logutils[i]->log_put_record(outfile, mod_buf);
                 if(ret < 0)
@@ -383,7 +385,7 @@ int main(int argc, char **argv)
 
                 memset(mod_buf, 0, DEF_MOD_BUF_SIZE);
             }
-        } while((ret = mod_logutils[i]->log_get_record(infile, mod_buf, &rec_id)) == 1);
+        } while((ret = mod_logutils[i]->log_get_record(infile, mod_buf)) == 1);
     }
 
     darshan_log_close(infile);


=====================================
darshan-util/darshan-hdf5-logutils.c
=====================================
--- a/darshan-util/darshan-hdf5-logutils.c
+++ b/darshan-util/darshan-hdf5-logutils.c
@@ -30,21 +30,21 @@ char *hdf5_f_counter_names[] = {
 };
 #undef X
 
-static int darshan_log_get_hdf5_file(darshan_fd fd, void* hdf5_buf,
-    darshan_record_id* rec_id);
+static int darshan_log_get_hdf5_file(darshan_fd fd, void* hdf5_buf);
 static int darshan_log_put_hdf5_file(darshan_fd fd, void* hdf5_buf);
 static void darshan_log_print_hdf5_file(void *file_rec,
     char *file_name, char *mnt_pt, char *fs_type);
+static void darshan_log_agg_hdf5_files(void *rec, void *agg_rec, int init_flag);
 
 struct darshan_mod_logutil_funcs hdf5_logutils =
 {
     .log_get_record = &darshan_log_get_hdf5_file,
     .log_put_record = &darshan_log_put_hdf5_file,
     .log_print_record = &darshan_log_print_hdf5_file,
+    .log_agg_records = &darshan_log_agg_hdf5_files
 };
 
-static int darshan_log_get_hdf5_file(darshan_fd fd, void* hdf5_buf,
-    darshan_record_id* rec_id)
+static int darshan_log_get_hdf5_file(darshan_fd fd, void* hdf5_buf)
 {
     struct darshan_hdf5_file *file;
     int i;
@@ -62,15 +62,14 @@ static int darshan_log_get_hdf5_file(darshan_fd fd, void* hdf5_buf,
         if(fd->swap_flag)
         {
             /* swap bytes if necessary */
-            DARSHAN_BSWAP64(&file->f_id);
-            DARSHAN_BSWAP64(&file->rank);
+            DARSHAN_BSWAP64(&(file->base_rec.id));
+            DARSHAN_BSWAP64(&(file->base_rec.rank));
             for(i=0; i<HDF5_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&file->counters[i]);
             for(i=0; i<HDF5_F_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&file->fcounters[i]);
         }
 
-        *rec_id = file->f_id;
         return(1);
     }
 }
@@ -98,20 +97,28 @@ static void darshan_log_print_hdf5_file(void *file_rec, char *file_name,
     for(i=0; i<HDF5_NUM_INDICES; i++)
     {
         DARSHAN_COUNTER_PRINT(darshan_module_names[DARSHAN_HDF5_MOD],
-            hdf5_file_rec->rank, hdf5_file_rec->f_id, hdf5_counter_names[i],
-            hdf5_file_rec->counters[i], file_name, mnt_pt, fs_type);
+            hdf5_file_rec->base_rec.rank, hdf5_file_rec->base_rec.id,
+            hdf5_counter_names[i], hdf5_file_rec->counters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     for(i=0; i<HDF5_F_NUM_INDICES; i++)
     {
         DARSHAN_F_COUNTER_PRINT(darshan_module_names[DARSHAN_HDF5_MOD],
-            hdf5_file_rec->rank, hdf5_file_rec->f_id, hdf5_f_counter_names[i],
-            hdf5_file_rec->fcounters[i], file_name, mnt_pt, fs_type);
+            hdf5_file_rec->base_rec.rank, hdf5_file_rec->base_rec.id,
+            hdf5_f_counter_names[i], hdf5_file_rec->fcounters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     return;
 }
 
+static void darshan_log_agg_hdf5_files(void *rec, void *agg_rec, int init_flag)
+{
+
+    return;
+}
+
 /*
  * Local variables:
  *  c-indent-level: 4


=====================================
darshan-util/darshan-logutils.h
=====================================
--- a/darshan-util/darshan-logutils.h
+++ b/darshan-util/darshan-logutils.h
@@ -59,8 +59,7 @@ struct darshan_mod_logutil_funcs
      */
     int (*log_get_record)(
         darshan_fd fd,
-        void* buf,
-        darshan_record_id* rec_id
+        void* buf
     );
     /* put a single module record into the log file.
      * return 0 on success, -1 on error
@@ -76,6 +75,12 @@ struct darshan_mod_logutil_funcs
         char *mnt_pt,
         char *fs_type
     );
+    /* combine two records into a single aggregate record */
+    void (*log_agg_records)(
+        void *rec,
+        void *agg_rec,
+        int init_flag
+    );
 };
 
 extern struct darshan_mod_logutil_funcs *mod_logutils[];


=====================================
darshan-util/darshan-mpiio-logutils.c
=====================================
--- a/darshan-util/darshan-mpiio-logutils.c
+++ b/darshan-util/darshan-mpiio-logutils.c
@@ -30,21 +30,21 @@ char *mpiio_f_counter_names[] = {
 };
 #undef X
 
-static int darshan_log_get_mpiio_file(darshan_fd fd, void* mpiio_buf,
-    darshan_record_id* rec_id);
+static int darshan_log_get_mpiio_file(darshan_fd fd, void* mpiio_buf);
 static int darshan_log_put_mpiio_file(darshan_fd fd, void* mpiio_buf);
 static void darshan_log_print_mpiio_file(void *file_rec,
     char *file_name, char *mnt_pt, char *fs_type);
+static void darshan_log_agg_mpiio_files(void *rec, void *agg_rec, int init_flag);
 
 struct darshan_mod_logutil_funcs mpiio_logutils =
 {
     .log_get_record = &darshan_log_get_mpiio_file,
     .log_put_record = &darshan_log_put_mpiio_file,
     .log_print_record = &darshan_log_print_mpiio_file,
+    .log_agg_records = &darshan_log_agg_mpiio_files
 };
 
-static int darshan_log_get_mpiio_file(darshan_fd fd, void* mpiio_buf,
-    darshan_record_id* rec_id)
+static int darshan_log_get_mpiio_file(darshan_fd fd, void* mpiio_buf)
 {
     struct darshan_mpiio_file *file;
     int i;
@@ -62,15 +62,14 @@ static int darshan_log_get_mpiio_file(darshan_fd fd, void* mpiio_buf,
         if(fd->swap_flag)
         {
             /* swap bytes if necessary */
-            DARSHAN_BSWAP64(&file->f_id);
-            DARSHAN_BSWAP64(&file->rank);
+            DARSHAN_BSWAP64(&(file->base_rec.id));
+            DARSHAN_BSWAP64(&(file->base_rec.rank));
             for(i=0; i<MPIIO_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&file->counters[i]);
             for(i=0; i<MPIIO_F_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&file->fcounters[i]);
         }
 
-        *rec_id = file->f_id;
         return(1);
     }
 }
@@ -98,20 +97,28 @@ static void darshan_log_print_mpiio_file(void *file_rec, char *file_name,
     for(i=0; i<MPIIO_NUM_INDICES; i++)
     {
         DARSHAN_COUNTER_PRINT(darshan_module_names[DARSHAN_MPIIO_MOD],
-            mpiio_file_rec->rank, mpiio_file_rec->f_id, mpiio_counter_names[i],
-            mpiio_file_rec->counters[i], file_name, mnt_pt, fs_type);
+            mpiio_file_rec->base_rec.rank, mpiio_file_rec->base_rec.id,
+            mpiio_counter_names[i], mpiio_file_rec->counters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     for(i=0; i<MPIIO_F_NUM_INDICES; i++)
     {
         DARSHAN_F_COUNTER_PRINT(darshan_module_names[DARSHAN_MPIIO_MOD],
-            mpiio_file_rec->rank, mpiio_file_rec->f_id, mpiio_f_counter_names[i],
-            mpiio_file_rec->fcounters[i], file_name, mnt_pt, fs_type);
+            mpiio_file_rec->base_rec.rank, mpiio_file_rec->base_rec.id,
+            mpiio_f_counter_names[i], mpiio_file_rec->fcounters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     return;
 }
 
+static void darshan_log_agg_mpiio_files(void *rec, void *agg_rec, int init_flag)
+{
+
+    return;
+}
+
 /*
  * Local variables:
  *  c-indent-level: 4


=====================================
darshan-util/darshan-null-logutils.c
=====================================
--- a/darshan-util/darshan-null-logutils.c
+++ b/darshan-util/darshan-null-logutils.c
@@ -32,11 +32,11 @@ char *null_f_counter_names[] = {
 #undef X
 
 /* prototypes for each of the NULL module's logutil functions */
-static int darshan_log_get_null_record(darshan_fd fd, void* null_buf,
-    darshan_record_id* rec_id);
+static int darshan_log_get_null_record(darshan_fd fd, void* null_buf);
 static int darshan_log_put_null_record(darshan_fd fd, void* null_buf);
 static void darshan_log_print_null_record(void *file_rec,
     char *file_name, char *mnt_pt, char *fs_type);
+static void darshan_log_agg_null_records(void *rec, void *agg_rec, int init_flag);
 
 /* structure storing each function needed for implementing the darshan
  * logutil interface. these functions are used for reading, writing, and
@@ -47,14 +47,14 @@ struct darshan_mod_logutil_funcs null_logutils =
     .log_get_record = &darshan_log_get_null_record,
     .log_put_record = &darshan_log_put_null_record,
     .log_print_record = &darshan_log_print_null_record,
+    .log_agg_records = &darshan_log_agg_null_records
 };
 
 /* retrieve a NULL record from log file descriptor 'fd', storing the
  * buffer in 'null_buf' and the corresponding Darshan record id in
  * 'rec_id'. Return 1 on successful record read, .
  */
-static int darshan_log_get_null_record(darshan_fd fd, void* null_buf, 
-    darshan_record_id* rec_id)
+static int darshan_log_get_null_record(darshan_fd fd, void* null_buf)
 {
     struct darshan_null_record *rec;
     int i;
@@ -74,16 +74,14 @@ static int darshan_log_get_null_record(darshan_fd fd, void* null_buf,
         if(fd->swap_flag)
         {
             /* swap bytes if necessary */
-            DARSHAN_BSWAP64(&rec->f_id);
-            DARSHAN_BSWAP64(&rec->rank);
+            DARSHAN_BSWAP64(&(rec->base_rec.id));
+            DARSHAN_BSWAP64(&(rec->base_rec.rank));
             for(i=0; i<NULL_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&rec->counters[i]);
             for(i=0; i<NULL_F_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&rec->fcounters[i]);
         }
 
-        /* set the output record id */
-        *rec_id = rec->f_id;
         return(1);
     }
 }
@@ -118,21 +116,29 @@ static void darshan_log_print_null_record(void *file_rec, char *file_name,
     {
         /* macro defined in darshan-logutils.h */
         DARSHAN_COUNTER_PRINT(darshan_module_names[DARSHAN_NULL_MOD],
-            null_rec->rank, null_rec->f_id, null_counter_names[i],
-            null_rec->counters[i], file_name, mnt_pt, fs_type);
+            null_rec->base_rec.rank, null_rec->base_rec.id,
+            null_counter_names[i], null_rec->counters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     for(i=0; i<NULL_F_NUM_INDICES; i++)
     {
         /* macro defined in darshan-logutils.h */
         DARSHAN_F_COUNTER_PRINT(darshan_module_names[DARSHAN_NULL_MOD],
-            null_rec->rank, null_rec->f_id, null_f_counter_names[i],
-            null_rec->fcounters[i], file_name, mnt_pt, fs_type);
+            null_rec->base_rec.rank, null_rec->base_rec.id,
+            null_f_counter_names[i], null_rec->fcounters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     return;
 }
 
+static void darshan_log_agg_null_records(void *rec, void *agg_rec, int init_flag)
+{
+
+    return;
+}
+
 /*
  * Local variables:
  *  c-indent-level: 4


=====================================
darshan-util/darshan-parser.c
=====================================
--- a/darshan-util/darshan-parser.c
+++ b/darshan-util/darshan-parser.c
@@ -209,6 +209,7 @@ int main(int argc, char **argv)
     char** mnt_pts;
     char** fs_types;
     time_t tmp_time = 0;
+    int64_t run_time = 0;
     char *token;
     char *save;
     char buffer[DARSHAN_JOB_METADATA_LEN];
@@ -287,7 +288,9 @@ int main(int argc, char **argv)
     tmp_time += job.end_time;
     printf("# end_time_asci: %s", ctime(&tmp_time));
     printf("# nprocs: %" PRId64 "\n", job.nprocs);
-    printf("# run time: %" PRId64 "\n", job.end_time - job.start_time + 1);
+    if(job.end_time >= job.start_time)
+        run_time = job.end_time - job.start_time + 1;
+    printf("# run time: %" PRId64 "\n", run_time);
     for(token=strtok_r(job.metadata, "\n", &save);
         token != NULL;
         token=strtok_r(NULL, "\n", &save))
@@ -348,7 +351,7 @@ int main(int argc, char **argv)
 
     for(i=0; i<DARSHAN_MAX_MODS; i++)
     {
-        darshan_record_id rec_id;
+        struct darshan_base_record *base_rec;
         void *save_io, *save_md;
 
         /* check each module for any data */
@@ -389,7 +392,7 @@ int main(int argc, char **argv)
             DARSHAN_PRINT_HEADER();
         }
 
-        ret = mod_logutils[i]->log_get_record(fd, mod_buf, &rec_id);
+        ret = mod_logutils[i]->log_get_record(fd, mod_buf);
         if(ret != 1)
         {
             fprintf(stderr, "Error: failed to parse the first %s module record.\n",
@@ -404,9 +407,10 @@ int main(int argc, char **argv)
             char *mnt_pt = NULL;
             char *fs_type = NULL;
             hash_entry_t *hfile = NULL;
+            base_rec = (struct darshan_base_record *)mod_buf;
 
             /* get the pathname for this record */
-            HASH_FIND(hlink, rec_hash, &rec_id, sizeof(darshan_record_id), ref);
+            HASH_FIND(hlink, rec_hash, &(base_rec->id), sizeof(darshan_record_id), ref);
             assert(ref);
 
             /* get mount point and fs type associated with this record */
@@ -437,7 +441,7 @@ int main(int argc, char **argv)
             if(i != DARSHAN_POSIX_MOD && i != DARSHAN_MPIIO_MOD)
                 continue;
 
-            HASH_FIND(hlink, file_hash, &rec_id, sizeof(darshan_record_id), hfile);
+            HASH_FIND(hlink, file_hash, &(base_rec->id), sizeof(darshan_record_id), hfile);
             if(!hfile)
             {
                 hfile = malloc(sizeof(*hfile));
@@ -449,14 +453,14 @@ int main(int argc, char **argv)
 
                 /* init */
                 memset(hfile, 0, sizeof(*hfile));
-                hfile->rec_id = rec_id;
+                hfile->rec_id = base_rec->id;
                 hfile->type = 0;
                 hfile->procs = 0;
                 hfile->rec_dat = NULL;
                 hfile->cumul_time = 0.0;
                 hfile->slowest_time = 0.0;
 
-                HASH_ADD(hlink, file_hash, rec_id, sizeof(darshan_record_id), hfile);
+                HASH_ADD(hlink, file_hash,rec_id, sizeof(darshan_record_id), hfile);
             }
 
             if(i == DARSHAN_POSIX_MOD)
@@ -474,7 +478,7 @@ int main(int argc, char **argv)
 
             memset(mod_buf, 0, DEF_MOD_BUF_SIZE);
 
-        } while((ret = mod_logutils[i]->log_get_record(fd, mod_buf, &rec_id)) == 1);
+        } while((ret = mod_logutils[i]->log_get_record(fd, mod_buf)) == 1);
         if (ret < 0)
         {
             ret = -1;
@@ -858,7 +862,7 @@ void mpiio_accum_file(struct darshan_mpiio_file *mfile,
 
     hfile->procs += 1;
 
-    if(mfile->rank == -1)
+    if(mfile->base_rec.rank == -1)
     {
         hfile->slowest_time = mfile->fcounters[MPIIO_F_SLOWEST_RANK_TIME];
     }
@@ -870,7 +874,7 @@ void mpiio_accum_file(struct darshan_mpiio_file *mfile,
             mfile->fcounters[MPIIO_F_WRITE_TIME]));
     }
 
-    if(mfile->rank == -1)
+    if(mfile->base_rec.rank == -1)
     {
         hfile->procs = nprocs;
         hfile->type |= FILETYPE_SHARED;
@@ -1111,7 +1115,7 @@ void mpiio_accum_perf(struct darshan_mpiio_file *mfile,
      *     by_slowest: use slowest rank time from log data
      *                 (most accurate but requires newer log version)
      */
-    if(mfile->rank == -1)
+    if(mfile->base_rec.rank == -1)
     {
         /* by_open */
         if(mfile->fcounters[MPIIO_F_CLOSE_TIMESTAMP] >
@@ -1162,11 +1166,12 @@ void mpiio_accum_perf(struct darshan_mpiio_file *mfile,
      */
     else
     {
-        pdata->rank_cumul_io_time[mfile->rank] +=
+        pdata->rank_cumul_io_time[mfile->base_rec.rank] +=
             (mfile->fcounters[MPIIO_F_META_TIME] +
             mfile->fcounters[MPIIO_F_READ_TIME] +
             mfile->fcounters[MPIIO_F_WRITE_TIME]);
-        pdata->rank_cumul_md_time[mfile->rank] += mfile->fcounters[MPIIO_F_META_TIME];
+        pdata->rank_cumul_md_time[mfile->base_rec.rank] +=
+            mfile->fcounters[MPIIO_F_META_TIME];
     }
 
     return;


=====================================
darshan-util/darshan-pnetcdf-logutils.c
=====================================
--- a/darshan-util/darshan-pnetcdf-logutils.c
+++ b/darshan-util/darshan-pnetcdf-logutils.c
@@ -30,21 +30,21 @@ char *pnetcdf_f_counter_names[] = {
 };
 #undef X
 
-static int darshan_log_get_pnetcdf_file(darshan_fd fd, void* pnetcdf_buf,
-    darshan_record_id* rec_id);
+static int darshan_log_get_pnetcdf_file(darshan_fd fd, void* pnetcdf_buf);
 static int darshan_log_put_pnetcdf_file(darshan_fd fd, void* pnetcdf_buf);
 static void darshan_log_print_pnetcdf_file(void *file_rec,
     char *file_name, char *mnt_pt, char *fs_type);
+static void darshan_log_agg_pnetcdf_files(void *rec, void *agg_rec, int init_flag);
 
 struct darshan_mod_logutil_funcs pnetcdf_logutils =
 {
     .log_get_record = &darshan_log_get_pnetcdf_file,
     .log_put_record = &darshan_log_put_pnetcdf_file,
     .log_print_record = &darshan_log_print_pnetcdf_file,
+    .log_agg_records = &darshan_log_agg_pnetcdf_files
 };
 
-static int darshan_log_get_pnetcdf_file(darshan_fd fd, void* pnetcdf_buf,
-    darshan_record_id* rec_id)
+static int darshan_log_get_pnetcdf_file(darshan_fd fd, void* pnetcdf_buf)
 {
     struct darshan_pnetcdf_file *file;
     int i;
@@ -62,15 +62,14 @@ static int darshan_log_get_pnetcdf_file(darshan_fd fd, void* pnetcdf_buf,
         if(fd->swap_flag)
         {
             /* swap bytes if necessary */
-            DARSHAN_BSWAP64(&file->f_id);
-            DARSHAN_BSWAP64(&file->rank);
+            DARSHAN_BSWAP64(&(file->base_rec.id));
+            DARSHAN_BSWAP64(&(file->base_rec.rank));
             for(i=0; i<PNETCDF_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&file->counters[i]);
             for(i=0; i<PNETCDF_F_NUM_INDICES; i++)
                 DARSHAN_BSWAP64(&file->fcounters[i]);
         }
 
-        *rec_id = file->f_id;
         return(1);
     }
 }
@@ -98,20 +97,28 @@ static void darshan_log_print_pnetcdf_file(void *file_rec, char *file_name,
     for(i=0; i<PNETCDF_NUM_INDICES; i++)
     {
         DARSHAN_COUNTER_PRINT(darshan_module_names[DARSHAN_PNETCDF_MOD],
-            pnetcdf_file_rec->rank, pnetcdf_file_rec->f_id, pnetcdf_counter_names[i],
-            pnetcdf_file_rec->counters[i], file_name, mnt_pt, fs_type);
+            pnetcdf_file_rec->base_rec.rank, pnetcdf_file_rec->base_rec.id,
+            pnetcdf_counter_names[i], pnetcdf_file_rec->counters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     for(i=0; i<PNETCDF_F_NUM_INDICES; i++)
     {
         DARSHAN_F_COUNTER_PRINT(darshan_module_names[DARSHAN_PNETCDF_MOD],
-            pnetcdf_file_rec->rank, pnetcdf_file_rec->f_id, pnetcdf_f_counter_names[i],
-            pnetcdf_file_rec->fcounters[i], file_name, mnt_pt, fs_type);
+            pnetcdf_file_rec->base_rec.rank, pnetcdf_file_rec->base_rec.id,
+            pnetcdf_f_counter_names[i], pnetcdf_file_rec->fcounters[i],
+            file_name, mnt_pt, fs_type);
     }
 
     return;
 }
 
+static void darshan_log_agg_pnetcdf_files(void *rec, void *agg_rec, int init_flag)
+{
+
+    return;
+}
+
 /*
  * Local variables:
  *  c-indent-level: 4


=====================================
darshan-util/darshan-posix-logutils.c
=====================================
--- a/darshan-util/darshan-posix-logutils.c
+++ b/darshan-util/darshan-posix-logutils.c
@@ -30,21 +30,21 @@ char *posix_f_counter_names[] = {
 };
 #undef X
 
-static int darshan_log_get_posix_file(darshan_fd fd, void* posix_buf,
-    darshan_record_id* rec_id);
+static int darshan_log_get_posix_file(darshan_fd fd, void* posix_buf);
 static int darshan_log_put_posix_file(darshan_fd fd, void* posix_buf);
 static void darshan_log_print_posix_file(void *file_rec,
     char *file_name, char *mnt_pt, char *fs_type);
+static void darshan_log_agg_posix_files(void *rec, void *agg_rec, int init_flag);
 
 struct darshan_mod_logutil_funcs posix_logutils =
 {
     .log_get_record = &darshan_log_get_posix_file,
     .log_put_record = &darshan_log_put_posix_file,
     .log_print_record = &darshan_log_print_posix_file,
+    .log_agg_records = &darshan_log_agg_posix_files
 };
 
-static int darshan_log_get_posix_file(darshan_fd fd, void* posix_buf, 
-    darshan_record_id* rec_id)
+static int darshan_log_get_posix_file(darshan_fd fd, void* posix_buf)
 {
     struct darshan_posix_file *file;
     int i;
@@ -70,7 +70,6 @@ static int darshan_log_get_posix_file(darshan_fd fd, void* posix_buf,
                 DARSHAN_BSWAP64(&file->fcounters[i]);
         }
 
-        *rec_id = file->base_rec.id;
         return(1);
     }
 }
@@ -114,6 +113,190 @@ static void darshan_log_print_posix_file(void *file_rec, char *file_name,
     return;
 }
 
+static void darshan_log_agg_posix_files(void *rec, void *agg_rec, int init_flag)
+{
+    struct darshan_posix_file *psx_rec = (struct darshan_posix_file *)rec;
+    struct darshan_posix_file *agg_psx_rec = (struct darshan_posix_file *)agg_rec;
+    int i;
+    double psx_time = psx_rec->fcounters[POSIX_F_READ_TIME] +
+        psx_rec->fcounters[POSIX_F_WRITE_TIME] +
+        psx_rec->fcounters[POSIX_F_META_TIME];
+
+    /* special case initialization of shared record for
+     * first call of this function
+     */
+    if(init_flag)
+    {
+        /* set fastest/slowest rank counters according to root rank.
+         * these counters will be determined as the aggregation progresses.
+         */
+        agg_psx_rec->counters[POSIX_FASTEST_RANK] = psx_rec->base_rec.rank;
+        agg_psx_rec->counters[POSIX_FASTEST_RANK_BYTES] =
+            psx_rec->counters[POSIX_BYTES_READ] +
+            psx_rec->counters[POSIX_BYTES_WRITTEN];
+        agg_psx_rec->fcounters[POSIX_F_FASTEST_RANK_TIME] = psx_time;
+
+        agg_psx_rec->counters[POSIX_SLOWEST_RANK] =
+            agg_psx_rec->counters[POSIX_FASTEST_RANK];
+        agg_psx_rec->counters[POSIX_SLOWEST_RANK_BYTES] =
+            agg_psx_rec->counters[POSIX_FASTEST_RANK_BYTES];
+        agg_psx_rec->fcounters[POSIX_F_SLOWEST_RANK_TIME] =
+            agg_psx_rec->fcounters[POSIX_F_FASTEST_RANK_TIME];
+    }
+
+    for(i = 0; i < POSIX_NUM_INDICES; i++)
+    {
+        switch(i)
+        {
+            case POSIX_OPENS:
+            case POSIX_READS:
+            case POSIX_WRITES:
+            case POSIX_SEEKS:
+            case POSIX_STATS:
+            case POSIX_MMAPS:
+            case POSIX_FOPENS:
+            case POSIX_FREADS:
+            case POSIX_FWRITES:
+            case POSIX_FSEEKS:
+            case POSIX_FSYNCS:
+            case POSIX_FDSYNCS:
+            case POSIX_BYTES_READ:
+            case POSIX_BYTES_WRITTEN:
+            case POSIX_CONSEC_READS:
+            case POSIX_CONSEC_WRITES:
+            case POSIX_SEQ_READS:
+            case POSIX_SEQ_WRITES:
+            case POSIX_RW_SWITCHES:
+            case POSIX_MEM_NOT_ALIGNED:
+            case POSIX_FILE_NOT_ALIGNED:
+            case POSIX_SIZE_READ_0_100:
+            case POSIX_SIZE_READ_100_1K:
+            case POSIX_SIZE_READ_1K_10K:
+            case POSIX_SIZE_READ_10K_100K:
+            case POSIX_SIZE_READ_100K_1M:
+            case POSIX_SIZE_READ_1M_4M:
+            case POSIX_SIZE_READ_4M_10M:
+            case POSIX_SIZE_READ_10M_100M:
+            case POSIX_SIZE_READ_100M_1G:
+            case POSIX_SIZE_READ_1G_PLUS:
+            case POSIX_SIZE_WRITE_0_100:
+            case POSIX_SIZE_WRITE_100_1K:
+            case POSIX_SIZE_WRITE_1K_10K:
+            case POSIX_SIZE_WRITE_10K_100K:
+            case POSIX_SIZE_WRITE_100K_1M:
+            case POSIX_SIZE_WRITE_1M_4M:
+            case POSIX_SIZE_WRITE_4M_10M:
+            case POSIX_SIZE_WRITE_10M_100M:
+            case POSIX_SIZE_WRITE_100M_1G:
+            case POSIX_SIZE_WRITE_1G_PLUS:
+                /* sum */
+                agg_psx_rec->counters[i] += psx_rec->counters[i];
+                break;
+            case POSIX_MODE:
+            case POSIX_MEM_ALIGNMENT:
+            case POSIX_FILE_ALIGNMENT:
+                /* just set to the input value */
+                agg_psx_rec->counters[i] = psx_rec->counters[i];
+                break;
+            case POSIX_MAX_BYTE_READ:
+            case POSIX_MAX_BYTE_WRITTEN:
+                /* max */
+                if(psx_rec->counters[i] > agg_psx_rec->counters[i])
+                {
+                    agg_psx_rec->counters[i] = psx_rec->counters[i];
+                }
+                break;
+            case POSIX_MAX_READ_TIME_SIZE:
+            case POSIX_MAX_WRITE_TIME_SIZE:
+            case POSIX_FASTEST_RANK:
+            case POSIX_FASTEST_RANK_BYTES:
+            case POSIX_SLOWEST_RANK:
+            case POSIX_SLOWEST_RANK_BYTES:
+                /* these are set with the FP counters */
+                break;
+            default:
+                /* TODO: common access counters and strides */
+                agg_psx_rec->counters[i] = -1;
+                break;
+        }
+    }
+
+    for(i = 0; i < POSIX_F_NUM_INDICES; i++)
+    {
+        switch(i)
+        {
+            case POSIX_F_READ_TIME:
+            case POSIX_F_WRITE_TIME:
+            case POSIX_F_META_TIME:
+                /* sum */
+                agg_psx_rec->fcounters[i] += psx_rec->fcounters[i];
+                break;
+            case POSIX_F_OPEN_TIMESTAMP:
+            case POSIX_F_READ_START_TIMESTAMP:
+            case POSIX_F_WRITE_START_TIMESTAMP:
+                /* minimum non-zero */
+                if((psx_rec->fcounters[i] > 0)  &&
+                    ((agg_psx_rec->fcounters[i] == 0) ||
+                    (psx_rec->fcounters[i] < agg_psx_rec->fcounters[i])))
+                {
+                    agg_psx_rec->fcounters[i] = psx_rec->fcounters[i];
+                }
+                break;
+            case POSIX_F_READ_END_TIMESTAMP:
+            case POSIX_F_WRITE_END_TIMESTAMP:
+            case POSIX_F_CLOSE_TIMESTAMP:
+                /* maximum */
+                if(psx_rec->fcounters[i] > agg_psx_rec->fcounters[i])
+                {
+                    agg_psx_rec->fcounters[i] = psx_rec->fcounters[i];
+                }
+                break;
+            case POSIX_F_MAX_READ_TIME:
+                if(psx_rec->fcounters[i] > agg_psx_rec->fcounters[i])
+                {
+                    agg_psx_rec->fcounters[i] = psx_rec->fcounters[i];
+                    agg_psx_rec->counters[POSIX_MAX_READ_TIME_SIZE] =
+                        psx_rec->counters[POSIX_MAX_READ_TIME_SIZE];
+                }
+                break;
+            case POSIX_F_MAX_WRITE_TIME:
+                if(psx_rec->fcounters[i] > agg_psx_rec->fcounters[i])
+                {
+                    agg_psx_rec->fcounters[i] = psx_rec->fcounters[i];
+                    agg_psx_rec->counters[POSIX_MAX_WRITE_TIME_SIZE] =
+                        psx_rec->counters[POSIX_MAX_WRITE_TIME_SIZE];
+                }
+                break;
+            case POSIX_F_FASTEST_RANK_TIME:
+                if(psx_time < agg_psx_rec->fcounters[POSIX_F_FASTEST_RANK_TIME])
+                {
+                    agg_psx_rec->counters[POSIX_FASTEST_RANK] = psx_rec->base_rec.rank;
+                    agg_psx_rec->counters[POSIX_FASTEST_RANK_BYTES] =
+                        psx_rec->counters[POSIX_BYTES_READ] +
+                        psx_rec->counters[POSIX_BYTES_WRITTEN];
+                    agg_psx_rec->fcounters[POSIX_F_FASTEST_RANK_TIME] = psx_time;
+                }
+                break;
+            case POSIX_F_SLOWEST_RANK_TIME:
+                if(psx_time > agg_psx_rec->fcounters[POSIX_F_SLOWEST_RANK_TIME])
+                {
+                    agg_psx_rec->counters[POSIX_SLOWEST_RANK] = psx_rec->base_rec.rank;
+                    agg_psx_rec->counters[POSIX_SLOWEST_RANK_BYTES] =
+                        psx_rec->counters[POSIX_BYTES_READ] +
+                        psx_rec->counters[POSIX_BYTES_WRITTEN];
+                    agg_psx_rec->fcounters[POSIX_F_SLOWEST_RANK_TIME] = psx_time;
+                }
+                break;
+            default:
+                /* TODO: variance */
+                agg_psx_rec->fcounters[i] = -1;
+                break;
+        }
+    }
+
+    return;
+}
+
 /*
  * Local variables:
  *  c-indent-level: 4


=====================================
darshan-util/darshan-stitch-logs.c
=====================================
--- a/darshan-util/darshan-stitch-logs.c
+++ b/darshan-util/darshan-stitch-logs.c
@@ -5,6 +5,8 @@
 #include <getopt.h>
 #include <glob.h>
 
+#include "uthash-1.9.2/src/uthash.h"
+
 #include "darshan-logutils.h"
 
 #define DEF_MOD_BUF_SIZE 1024 /* 1 KiB is enough for all current mod records ... */
@@ -14,6 +16,16 @@
 
 /* TODO: how do we set the output logfile name to be unique, and have necessary semantic info contained */
 
+/* TODO: set job end timestamp? */
+
+struct darshan_shared_record_ref
+{
+    darshan_record_id id;
+    int ref_cnt;
+    char agg_rec[DEF_MOD_BUF_SIZE];
+    UT_hash_handle hlink;
+};
+
 void usage(char *exename)
 {
     fprintf(stderr, "Usage: %s [options] <tmp_dir> <job_id>\n", exename);
@@ -87,6 +99,94 @@ int logfile_path_comp(const void *a, const void *b)
         return(0);
 }
 
+int build_mod_shared_rec_hash(glob_t *globbuf, darshan_module_id mod_id,
+    int nprocs, char *mod_buf, struct darshan_shared_record_ref **shared_rec_hash)
+{
+    darshan_fd in_fd;
+    struct darshan_base_record *base_rec;
+    struct darshan_shared_record_ref *ref, *tmp;
+    int init = 0;
+    int ret;
+    int i;
+
+    /* loop over each input log file */
+    for(i = 0; i < globbuf->gl_pathc; i++)
+    {
+        in_fd = darshan_log_open(globbuf->gl_pathv[i]);
+        if(in_fd == NULL)
+        {
+            fprintf(stderr,
+                "Error: unable to open input Darshan log file %s.\n",
+                globbuf->gl_pathv[i]);
+            return(-1);
+        }
+
+        while((ret = mod_logutils[mod_id]->log_get_record(in_fd, mod_buf)) == 1)
+        {
+            base_rec = (struct darshan_base_record *)mod_buf;
+
+            /* initialize the hash with the first rank's records */
+            if(!init)
+            {
+                struct darshan_base_record *agg_base;
+
+                /* create a new ref and add to the hash */
+                ref = malloc(sizeof(*ref));
+                if(!ref)
+                {
+                    darshan_log_close(in_fd);
+                    return(-1);
+                }
+
+                /* initialize the aggregate record with this rank's record */
+                agg_base = (struct darshan_base_record *)ref->agg_rec;
+                agg_base->id = base_rec->id;
+                agg_base->rank = -1;
+                mod_logutils[mod_id]->log_agg_records(mod_buf, ref->agg_rec, 1);
+
+                ref->id = base_rec->id;
+                ref->ref_cnt = 1;
+                HASH_ADD(hlink, *shared_rec_hash, id, sizeof(darshan_record_id), ref);
+                init = 1;
+            }
+            else
+            {
+                /* search for this record in shared record hash */
+                HASH_FIND(hlink, *shared_rec_hash, &(base_rec->id),
+                    sizeof(darshan_record_id), ref);
+                if(ref)
+                {
+                    /* if found, aggregate this rank's record into the shared record */
+                    mod_logutils[mod_id]->log_agg_records(mod_buf, ref->agg_rec, 0);
+                    ref->ref_cnt++;
+                }
+            }
+        }
+        if(ret < 0)
+        {
+            fprintf(stderr,
+                "Error: unable to read %s module record from input log file %s.\n",
+                darshan_module_names[mod_id], globbuf->gl_pathv[i]);
+            darshan_log_close(in_fd);
+            return(-1);
+        }
+
+        darshan_log_close(in_fd);
+    }
+
+    /* prune any non-shared records from the hash one last time */
+    HASH_ITER(hlink, *shared_rec_hash, ref, tmp)
+    {
+        if(ref->ref_cnt != nprocs)
+        {
+            HASH_DELETE(hlink, *shared_rec_hash, ref);
+            free(ref);
+        }
+    }
+
+    return(0);
+}
+
 int main(int argc, char *argv[])
 {
     int shared_redux;
@@ -104,8 +204,10 @@ int main(int argc, char *argv[])
     struct darshan_record_ref *in_hash = NULL;
     struct darshan_record_ref *stitch_hash = NULL;
     struct darshan_record_ref *ref, *tmp, *found;
-    darshan_record_id rec_id;
-    char *mod_buf;
+    struct darshan_shared_record_ref *shared_rec_hash = NULL;
+    struct darshan_shared_record_ref *sref, *stmp;
+    struct darshan_base_record *base_rec;
+    char mod_buf[DEF_MOD_BUF_SIZE];
     int i, j;
     int ret;
 
@@ -161,6 +263,21 @@ int main(int argc, char *argv[])
             return(-1);
         }
 
+        /* if the input darshan log has metadata set indicating the darshan
+         * shutdown procedure was called on the log, then we error out. if the
+         * shutdown procedure was started, then it's possible the log has
+         * incomplete or corrupt data, so we just throw out the data for now.
+         */
+        if(strstr(in_job.metadata, "darshan_shutdown=yes"))
+        {
+            fprintf(stderr,
+                "Error: potentially corrupt data found in input log file %s.\n",
+                globbuf.gl_pathv[i]);
+            darshan_log_close(in_fd);
+            globfree(&globbuf);
+            return(-1);
+        }
+
         if(i == 0)
         {
             /* get job data, exe, & mounts directly from the first input log */
@@ -198,6 +315,7 @@ int main(int argc, char *argv[])
                 stitch_job.end_time = in_job.end_time;
         }
 
+        /* read the hash of ids->names for the input log */
         ret = darshan_log_gethash(in_fd, &in_hash);
         if(ret < 0)
         {
@@ -283,16 +401,6 @@ int main(int argc, char *argv[])
         return(-1);
     }
 
-    mod_buf = malloc(DEF_MOD_BUF_SIZE);
-    if(!mod_buf)
-    {
-        globfree(&globbuf);
-        darshan_log_close(stitch_fd);
-        unlink(stitch_logname);
-        return(-1);
-    }
-    memset(mod_buf, 0, DEF_MOD_BUF_SIZE);
-
     /* iterate over active darshan modules and gather module data to write
      * to the stitched together output log
      */
@@ -302,21 +410,37 @@ int main(int argc, char *argv[])
 
         if(shared_redux)
         {
-            /* copy all root's file records into an array */
-
-            /* compare and updated shared records? */
-            for(j = 1; j < globbuf.gl_pathc; j++)
+            /* build the hash of records shared globally by this module */
+            ret = build_mod_shared_rec_hash(&globbuf, i, stitch_job.nprocs,
+                mod_buf, &shared_rec_hash);
+            if(ret < 0)
             {
+                fprintf(stderr,
+                    "Error: unable to build list of %s module's shared records.\n",
+                    darshan_module_names[i]);
+                globfree(&globbuf);
+                darshan_log_close(stitch_fd);
+                unlink(stitch_logname);
+                return(-1);
             }
 
-            /* XXX aggregate shared records? */
-            for(j = 0; j < globbuf.gl_pathc; j++)
+            /* write out the shared records first */
+            HASH_ITER(hlink, shared_rec_hash, sref, stmp)
             {
-
+                ret = mod_logutils[i]->log_put_record(stitch_fd, sref->agg_rec);
+                if(ret < 0)
+                {
+                    fprintf(stderr,
+                        "Error: unable to write %s module record to output darshan log.\n",
+                        darshan_module_names[i]);
+                    globfree(&globbuf);
+                    darshan_log_close(stitch_fd);
+                    unlink(stitch_logname);
+                    return(-1);
+                }
             }
         }
 
-        /* XXX third write each rank's blobs, with rank 0 writing the shared ones? */
         for(j = 0; j < globbuf.gl_pathc; j++)
         {
             in_fd = darshan_log_open(globbuf.gl_pathv[j]);
@@ -325,48 +449,63 @@ int main(int argc, char *argv[])
                 fprintf(stderr,
                     "Error: unable to open input Darshan log file %s.\n",
                     globbuf.gl_pathv[j]);
-                free(mod_buf);
                 globfree(&globbuf);
                 darshan_log_close(in_fd);
+                darshan_log_close(stitch_fd);
                 unlink(stitch_logname);
                 return(-1);
             }
 
             /* loop over module records and write them to output file */
-            while((ret = mod_logutils[i]->log_get_record(in_fd, mod_buf, &rec_id)) == 1)
+            while((ret = mod_logutils[i]->log_get_record(in_fd, mod_buf)) == 1)
             {
+                base_rec = (struct darshan_base_record *)mod_buf;
+
+                HASH_FIND(hlink, shared_rec_hash, &(base_rec->id), sizeof(darshan_record_id), sref);
+                if(sref)
+                    continue; /* skip shared records */
+
                 ret = mod_logutils[i]->log_put_record(stitch_fd, mod_buf);
                 if(ret < 0)
                 {
                     fprintf(stderr,
                         "Error: unable to write %s module record to output log file %s.\n",
                         darshan_module_names[i], globbuf.gl_pathv[j]);
-                    free(mod_buf);
+                    globfree(&globbuf);
                     darshan_log_close(in_fd);
+                    darshan_log_close(stitch_fd);
                     unlink(stitch_logname);
                     return(-1);
                 }
-
-                memset(mod_buf, 0, DEF_MOD_BUF_SIZE);
             }
             if(ret < 0)
             {
                 fprintf(stderr,
                     "Error: unable to read %s module record from input log file %s.\n",
                     darshan_module_names[i], globbuf.gl_pathv[j]);
-                free(mod_buf);
+                globfree(&globbuf);
                 darshan_log_close(in_fd);
+                darshan_log_close(stitch_fd);
                 unlink(stitch_logname);
                 return(-1);
             }
 
             darshan_log_close(in_fd);
         }
+
+        /* clear the shared record hash for the next module */
+        if(shared_redux)
+        {
+            HASH_ITER(hlink, shared_rec_hash, sref, stmp)
+            {
+                HASH_DELETE(hlink, shared_rec_hash, sref);
+                free(sref);
+            }
+        }
     }
 
     darshan_log_close(stitch_fd);
     globfree(&globbuf);
-    free(mod_buf);
 
     return(0);
 }



View it on GitLab: https://xgitlab.cels.anl.gov/darshan/darshan/compare/ed103a2ae7a9c396eb0a21642de5c3c9db5de18a...80c07c27781a9680cfa7c001badc2ebe53e6dd00
-------------- next part --------------
An HTML attachment was scrubbed...
URL: <http://lists.mcs.anl.gov/pipermail/darshan-commits/attachments/20151217/221ab3df/attachment-0001.html>


More information about the Darshan-commits mailing list