[Darshan-commits] [Git][darshan/darshan][dev-fork-safe] use pthread_atfork to gracefully handle fork
Shane Snyder
xgitlab at cels.anl.gov
Thu Feb 4 12:01:37 CST 2021
Shane Snyder pushed to branch dev-fork-safe at darshan / darshan
Commits:
2b1b38f2 by Shane Snyder at 2021-02-04T12:00:58-06:00
use pthread_atfork to gracefully handle fork
this required reworking Darshan module interface to seprarate
shutdown into 2 steps: module output and module cleanup
- - - - -
14 changed files:
- darshan-runtime/darshan-core.h
- darshan-runtime/darshan.h
- darshan-runtime/lib/darshan-bgq.c
- darshan-runtime/lib/darshan-core-init-finalize.c
- darshan-runtime/lib/darshan-core.c
- darshan-runtime/lib/darshan-dxt.c
- darshan-runtime/lib/darshan-hdf5.c
- darshan-runtime/lib/darshan-lustre.c
- darshan-runtime/lib/darshan-mdhim.c
- darshan-runtime/lib/darshan-mpiio.c
- darshan-runtime/lib/darshan-null.c
- darshan-runtime/lib/darshan-pnetcdf.c
- darshan-runtime/lib/darshan-posix.c
- darshan-runtime/lib/darshan-stdio.c
Changes:
=====================================
darshan-runtime/darshan-core.h
=====================================
@@ -122,7 +122,7 @@ struct darshan_core_runtime
};
void darshan_core_initialize(int argc, char **argv);
-void darshan_core_shutdown(void);
+void darshan_core_shutdown(int write_log);
uint32_t darshan_hashlittle(const void *key, size_t length, uint32_t initval);
uint64_t darshan_hash(const register unsigned char *k, register uint64_t length, register uint64_t level);
=====================================
darshan-runtime/darshan.h
=====================================
@@ -95,20 +95,27 @@ typedef void (*darshan_module_redux)(
);
#endif
/*
- * module developers _must_ define a 'darshan_module_shutdown' function
+ * module developers _must_ define a 'darshan_module_output' function
* for allowing darshan-core to call into a module and retrieve final
* output data to be saved in the log.
*/
-typedef void (*darshan_module_shutdown)(
+typedef void (*darshan_module_output)(
void **mod_buf, /* output parameter to save module buffer address */
int *mod_buf_sz /* output parameter to save module buffer size */
);
+/*
+ * module developers _must_ define a 'darshan_module_shutdown' function
+ * for allowing darshan-core to call into a module and retrieve final
+ * output data to be saved in the log.
+ */
+typedef void (*darshan_module_cleanup)(void);
typedef struct darshan_module_funcs
{
#ifdef HAVE_MPI
darshan_module_redux mod_redux_func;
#endif
- darshan_module_shutdown mod_shutdown_func;
+ darshan_module_output mod_output_func;
+ darshan_module_cleanup mod_cleanup_func;
} darshan_module_funcs;
/* stores FS info from statfs calls for a given mount point */
=====================================
darshan-runtime/lib/darshan-bgq.c
=====================================
@@ -55,9 +55,11 @@ static void bgq_mpi_redux(
MPI_Comm mod_comm,
darshan_record_id *shared_recs,
int shared_rec_count);
-static void bgq_shutdown(
+static void bgq_output(
void **buffer,
int *size);
+static void bgq_cleanup(
+ void);
/* macros for obtaining/releasing the BGQ module lock */
#define BGQ_LOCK() pthread_mutex_lock(&bgq_runtime_mutex)
@@ -113,7 +115,8 @@ void bgq_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &bgq_mpi_redux,
#endif
- .mod_shutdown_func = &bgq_shutdown
+ .mod_output_func = &bgq_output,
+ .mod_output_func = &bgq_cleanup
};
BGQ_LOCK();
@@ -187,7 +190,7 @@ static int cmpr(const void *p1, const void *p2)
}
/********************************************************************************
- * shutdown function exported by this module for coordinating with darshan-core *
+ * functions exported by this module for coordinating with darshan-core *
********************************************************************************/
static void bgq_mpi_redux(
@@ -264,11 +267,19 @@ static void bgq_shutdown(
*size = 0;
}
+ BGQ_UNLOCK();
+ return;
+}
+
+static void bgq_cleanup()
+{
+ BGQ_LOCK();
+ assert(bgq_runtime);
+
free(bgq_runtime);
bgq_runtime = NULL;
BGQ_UNLOCK();
-
return;
}
=====================================
darshan-runtime/lib/darshan-core-init-finalize.c
=====================================
@@ -82,7 +82,7 @@ int DARSHAN_DECL(MPI_Finalize)(void)
MAP_OR_FAIL(PMPI_Finalize);
- darshan_core_shutdown();
+ darshan_core_shutdown(1);
ret = __real_PMPI_Finalize();
return(ret);
@@ -108,7 +108,7 @@ __attribute__((destructor)) void serial_finalize(void)
char *no_mpi;
no_mpi = getenv(DARSHAN_ENABLE_NONMPI);
if (no_mpi)
- darshan_core_shutdown();
+ darshan_core_shutdown(1);
return;
}
#endif
=====================================
darshan-runtime/lib/darshan-core.c
=====================================
@@ -55,6 +55,8 @@ static int my_rank = 0;
static int nprocs = 1;
static int darshan_mem_alignment = 1;
static long darshan_mod_mem_quota = DARSHAN_MOD_MEM_MAX;
+static int orig_parent_pid = 0;
+static int parent_pid;
static struct darshan_core_mnt_data mnt_data_array[DARSHAN_MAX_MNTS];
static int mnt_data_count = 0;
@@ -154,6 +156,7 @@ static int darshan_deflate_buffer(
static void darshan_core_cleanup(
struct darshan_core_runtime* core);
static double darshan_core_wtime_absolute(void);
+static void darshan_core_fork_child_cb(void);
#define DARSHAN_CORE_LOCK() pthread_mutex_lock(&darshan_core_mutex)
#define DARSHAN_CORE_UNLOCK() pthread_mutex_unlock(&darshan_core_mutex)
@@ -175,7 +178,7 @@ static double darshan_core_wtime_absolute(void);
if(log_created) \
unlink(logfile_name); \
} \
- goto exit; \
+ goto cleanup; \
} \
} while(0)
@@ -187,7 +190,7 @@ static double darshan_core_wtime_absolute(void);
DARSHAN_WARN(__err_str, ## __VA_ARGS__); \
if(log_created) \
unlink(logfile_name); \
- goto exit; \
+ goto cleanup; \
} \
} while(0)
@@ -260,7 +263,10 @@ void darshan_core_initialize(int argc, char **argv)
if(!jobid_str || ret != 1)
{
/* use pid as fall back */
- jobid = init_pid;
+ if(!orig_parent_pid)
+ jobid = init_pid;
+ else
+ jobid = orig_parent_pid;
}
/* set the memory quota for darshan modules' records */
@@ -291,6 +297,12 @@ void darshan_core_initialize(int argc, char **argv)
}
#endif
+ /* setup fork handlers if not using MPI */
+ if(!using_mpi && !orig_parent_pid)
+ {
+ pthread_atfork(NULL, NULL, &darshan_core_fork_child_cb);
+ }
+
/* record absolute start time at startup so that we can later
* generate relative times with this as a reference point.
*/
@@ -417,7 +429,7 @@ void darshan_core_initialize(int argc, char **argv)
return;
}
-void darshan_core_shutdown()
+void darshan_core_shutdown(int write_log)
{
struct darshan_core_runtime *final_core;
double start_log_time;
@@ -434,11 +446,15 @@ void darshan_core_shutdown()
char *logfile_name = NULL;
darshan_core_log_fh log_fh;
int log_created = 0;
- int shutdown_pid = getpid();
int meta_remain = 0;
char *m;
int i;
int ret;
+#ifdef HAVE_MPI
+ darshan_record_id *shared_recs = NULL;
+ darshan_record_id *mod_shared_recs = NULL;
+ int shared_rec_cnt = 0;
+#endif
/* disable darhan-core while we shutdown */
DARSHAN_CORE_LOCK();
@@ -451,6 +467,10 @@ void darshan_core_shutdown()
darshan_core = NULL;
DARSHAN_CORE_UNLOCK();
+ /* skip to cleanup if not writing a log */
+ if(!write_log)
+ goto cleanup;
+
/* grab some initial timing information */
#ifdef HAVE_MPI
/* if using mpi, sync across procs first */
@@ -476,7 +496,7 @@ void darshan_core_shutdown()
final_core->comp_buf = malloc(darshan_mod_mem_quota);
logfile_name = malloc(PATH_MAX);
if(!final_core->comp_buf || !logfile_name)
- goto exit;
+ goto cleanup;
/* set which modules were used locally */
for(i = 0; i < DARSHAN_MAX_MODS; i++)
@@ -486,10 +506,6 @@ void darshan_core_shutdown()
}
#ifdef HAVE_MPI
- darshan_record_id *shared_recs = NULL;
- darshan_record_id *mod_shared_recs = NULL;
- int shared_rec_cnt = 0;
-
if(using_mpi)
{
/* allreduce locally active mods to determine globally active mods */
@@ -522,12 +538,12 @@ void darshan_core_shutdown()
}
#endif
- /* detect whether our PID changed since init, which is caused by fork() */
+ /* detect whether we forked, saving the parent pid in the log metadata if so */
/* NOTE: this should only be triggered in non-MPI cases, since MPI mode still
* bootstraps the shutdown procedure on MPI_Finalize, which forked processes
* will not call
*/
- if(shutdown_pid != final_core->pid)
+ if(orig_parent_pid)
{
/* set fork metadata */
meta_remain = DARSHAN_JOB_METADATA_LEN -
@@ -536,9 +552,8 @@ void darshan_core_shutdown()
{
m = final_core->log_job_p->metadata +
strlen(final_core->log_job_p->metadata);
- sprintf(m, "fork_parent=%d\n", final_core->pid);
+ sprintf(m, "fork_parent=%d\n", parent_pid);
}
- final_core->pid = shutdown_pid;
}
/* get the log file name */
@@ -546,7 +561,7 @@ void darshan_core_shutdown()
if(strlen(logfile_name) == 0)
{
/* failed to generate log file name */
- goto exit;
+ goto cleanup;
}
if(internal_timing_flag)
@@ -636,7 +651,7 @@ void darshan_core_shutdown()
#endif
/* get the final output buffer */
- this_mod->mod_funcs.mod_shutdown_func(&mod_buf, &mod_buf_sz);
+ this_mod->mod_funcs.mod_output_func(&mod_buf, &mod_buf_sz);
}
/* append this module's data to the darshan log */
@@ -645,12 +660,6 @@ void darshan_core_shutdown()
final_core->log_hdr_p->mod_map[i].len =
gz_fp - final_core->log_hdr_p->mod_map[i].off;
- /* XXX: DXT manages its own module memory buffers, so we need to
- * explicitly free them
- */
- if(i == DXT_POSIX_MOD || i == DXT_MPIIO_MOD)
- free(mod_buf);
-
if(internal_timing_flag)
mod2[i] = darshan_core_wtime();
@@ -727,7 +736,7 @@ void darshan_core_shutdown()
MPI_DOUBLE, MPI_MAX, 0, final_core->mpi_comm);
/* let rank 0 report the timing info */
- goto exit;
+ goto cleanup;
}
}
#endif
@@ -746,7 +755,11 @@ void darshan_core_shutdown()
darshan_core_fprintf(stderr, "darshan:core_shutdown\t%d\t%f\n", nprocs, all_tm);
}
-exit:
+cleanup:
+ for(i = 0; i < DARSHAN_MAX_MODS; i++)
+ if(final_core->mod_array[i])
+ final_core->mod_array[i]->mod_funcs.mod_cleanup_func();
+ darshan_core_cleanup(final_core);
#ifdef HAVE_MPI
if(using_mpi)
{
@@ -755,7 +768,6 @@ exit:
}
#endif
free(logfile_name);
- darshan_core_cleanup(final_core);
return;
}
@@ -2074,6 +2086,22 @@ static void darshan_core_cleanup(struct darshan_core_runtime* core)
return;
}
+static void darshan_core_fork_child_cb(void)
+{
+ /* hold onto the original parent PID, which we will use as jobid if the user didn't
+ * provide a jobid env variable
+ */
+ parent_pid = darshan_core->pid;
+ if(!orig_parent_pid)
+ orig_parent_pid = parent_pid;
+
+ /* shutdown and re-init darshan, making sure to not write out a log file */
+ darshan_core_shutdown(0);
+ darshan_core_initialize(0, NULL);
+
+ return;
+}
+
/* crude benchmarking hook into darshan-core to benchmark Darshan
* shutdown overhead using a variety of application I/O workloads
*/
@@ -2099,7 +2127,7 @@ void darshan_shutdown_bench(int argc, char **argv)
if(my_rank == 0)
fprintf(stderr, "# 1 unique file per proc\n");
PMPI_Barrier(MPI_COMM_WORLD);
- darshan_core_shutdown();
+ darshan_core_shutdown(1);
darshan_core = NULL;
sleep(1);
@@ -2114,7 +2142,7 @@ void darshan_shutdown_bench(int argc, char **argv)
if(my_rank == 0)
fprintf(stderr, "# 1 shared file per proc\n");
PMPI_Barrier(MPI_COMM_WORLD);
- darshan_core_shutdown();
+ darshan_core_shutdown(1);
darshan_core = NULL;
sleep(1);
@@ -2129,7 +2157,7 @@ void darshan_shutdown_bench(int argc, char **argv)
if(my_rank == 0)
fprintf(stderr, "# 1024 unique files per proc\n");
PMPI_Barrier(MPI_COMM_WORLD);
- darshan_core_shutdown();
+ darshan_core_shutdown(1);
darshan_core = NULL;
sleep(1);
@@ -2144,7 +2172,7 @@ void darshan_shutdown_bench(int argc, char **argv)
if(my_rank == 0)
fprintf(stderr, "# 1024 shared files per proc\n");
PMPI_Barrier(MPI_COMM_WORLD);
- darshan_core_shutdown();
+ darshan_core_shutdown(1);
darshan_core = NULL;
sleep(1);
=====================================
darshan-runtime/lib/darshan-dxt.c
=====================================
@@ -149,16 +149,16 @@ static struct dxt_file_record_ref *dxt_posix_track_new_file_record(
darshan_record_id rec_id);
static struct dxt_file_record_ref *dxt_mpiio_track_new_file_record(
darshan_record_id rec_id);
-static void dxt_posix_cleanup_runtime(
- void);
-static void dxt_mpiio_cleanup_runtime(
- void);
-/* DXT shutdown routines for darshan-core */
-static void dxt_posix_shutdown(
+/* DXT output/cleanup routines for darshan-core */
+static void dxt_posix_output(
void **dxt_buf, int *dxt_buf_sz);
-static void dxt_mpiio_shutdown(
+static void dxt_mpiio_output(
void **dxt_buf, int *dxt_buf_sz);
+static void dxt_posix_cleanup(
+ void);
+static void dxt_mpiio_cleanup(
+ void);
static struct dxt_posix_runtime *dxt_posix_runtime = NULL;
static struct dxt_mpiio_runtime *dxt_mpiio_runtime = NULL;
@@ -278,7 +278,8 @@ void dxt_posix_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = NULL,
#endif
- .mod_shutdown_func = &dxt_posix_shutdown
+ .mod_output_func = &dxt_posix_output,
+ .mod_cleanup_func = &dxt_posix_cleanup
};
int ret;
double tmpfloat;
@@ -346,7 +347,8 @@ void dxt_mpiio_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = NULL,
#endif
- .mod_shutdown_func = &dxt_mpiio_shutdown
+ .mod_output_func = &dxt_mpiio_output,
+ .mod_cleanup_func = &dxt_mpiio_cleanup
};
int ret;
double tmpfloat;
@@ -976,32 +978,8 @@ static void dxt_free_record_data(void *rec_ref_p, void *user_ptr)
free(dxt_rec_ref->file_rec);
}
-static void dxt_posix_cleanup_runtime()
-{
- darshan_iter_record_refs(dxt_posix_runtime->rec_id_hash,
- dxt_free_record_data, NULL);
- darshan_clear_record_refs(&(dxt_posix_runtime->rec_id_hash), 1);
-
- free(dxt_posix_runtime);
- dxt_posix_runtime = NULL;
-
- return;
-}
-
-static void dxt_mpiio_cleanup_runtime()
-{
- darshan_iter_record_refs(dxt_mpiio_runtime->rec_id_hash,
- dxt_free_record_data, NULL);
- darshan_clear_record_refs(&(dxt_mpiio_runtime->rec_id_hash), 1);
-
- free(dxt_mpiio_runtime);
- dxt_mpiio_runtime = NULL;
-
- return;
-}
-
/********************************************************************************
- * shutdown function exported by this module for coordinating with darshan-core *
+ * functions exported by this module for coordinating with darshan-core *
********************************************************************************/
static void dxt_serialize_posix_records(void *rec_ref_p, void *user_ptr)
@@ -1087,7 +1065,7 @@ static void dxt_serialize_posix_records(void *rec_ref_p, void *user_ptr)
#endif
}
-static void dxt_posix_shutdown(
+static void dxt_posix_output(
void **dxt_posix_buf,
int *dxt_posix_buf_sz)
{
@@ -1109,8 +1087,22 @@ static void dxt_posix_shutdown(
*dxt_posix_buf = dxt_posix_runtime->record_buf;
*dxt_posix_buf_sz = dxt_posix_runtime->record_buf_size;
+ return;
+}
+
+static void dxt_posix_cleanup()
+{
+ assert(dxt_posix_runtime);
+
+ free(dxt_posix_runtime->record_buf);
+
/* shutdown internal structures used for instrumenting */
- dxt_posix_cleanup_runtime();
+ darshan_iter_record_refs(dxt_posix_runtime->rec_id_hash,
+ dxt_free_record_data, NULL);
+ darshan_clear_record_refs(&(dxt_posix_runtime->rec_id_hash), 1);
+
+ free(dxt_posix_runtime);
+ dxt_posix_runtime = NULL;
return;
}
@@ -1195,7 +1187,7 @@ static void dxt_serialize_mpiio_records(void *rec_ref_p, void *user_ptr)
#endif
}
-static void dxt_mpiio_shutdown(
+static void dxt_mpiio_output(
void **dxt_mpiio_buf,
int *dxt_mpiio_buf_sz)
{
@@ -1217,8 +1209,22 @@ static void dxt_mpiio_shutdown(
*dxt_mpiio_buf = dxt_mpiio_runtime->record_buf;
*dxt_mpiio_buf_sz = dxt_mpiio_runtime->record_buf_size;
+ return;
+}
+
+static void dxt_mpiio_cleanup()
+{
+ assert(dxt_mpiio_runtime);
+
+ free(dxt_mpiio_runtime->record_buf);
+
/* shutdown internal structures used for instrumenting */
- dxt_mpiio_cleanup_runtime();
+ darshan_iter_record_refs(dxt_mpiio_runtime->rec_id_hash,
+ dxt_free_record_data, NULL);
+ darshan_clear_record_refs(&(dxt_mpiio_runtime->rec_id_hash), 1);
+
+ free(dxt_mpiio_runtime);
+ dxt_mpiio_runtime = NULL;
return;
}
=====================================
darshan-runtime/lib/darshan-hdf5.c
=====================================
@@ -81,10 +81,6 @@ static struct hdf5_dataset_record_ref *hdf5_track_new_dataset_record(
darshan_record_id rec_id, const char *rec_name);
static void hdf5_finalize_dataset_records(
void *rec_ref_p, void *user_ptr);
-static void hdf5_cleanup_file_runtime(
- void);
-static void hdf5_cleanup_dataset_runtime(
- void);
#ifdef HAVE_MPI
static void hdf5_file_record_reduction_op(
void* inrec_v, void* inoutrec_v, int *len, MPI_Datatype *datatype);
@@ -100,10 +96,14 @@ static void hdf5_dataset_mpi_redux(
void *hdf5_buf, MPI_Comm mod_comm,
darshan_record_id *shared_recs, int shared_rec_count);
#endif
-static void hdf5_file_shutdown(
+static void hdf5_file_output(
void **hdf5_buf, int *hdf5_buf_sz);
-static void hdf5_dataset_shutdown(
+static void hdf5_dataset_output(
void **hdf5_buf, int *hdf5_buf_sz);
+static void hdf5_file_cleanup(
+ void);
+static void hdf5_dataset_cleanup(
+ void);
static struct hdf5_runtime *hdf5_file_runtime = NULL;
static struct hdf5_runtime *hdf5_dataset_runtime = NULL;
@@ -882,7 +882,8 @@ static void hdf5_file_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &hdf5_file_mpi_redux,
#endif
- .mod_shutdown_func = &hdf5_file_shutdown
+ .mod_output_func = &hdf5_file_output,
+ .mod_cleanup_func = &hdf5_file_cleanup
};
/* try and store the default number of records for this module */
@@ -921,7 +922,8 @@ static void hdf5_dataset_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &hdf5_dataset_mpi_redux,
#endif
- .mod_shutdown_func = &hdf5_dataset_shutdown
+ .mod_output_func = &hdf5_dataset_output,
+ .mod_cleanup_func = &hdf5_dataset_cleanup
};
/* try and store the default number of records for this module */
@@ -1063,28 +1065,6 @@ static void hdf5_finalize_dataset_records(void *rec_ref_p, void *user_ptr)
return;
}
-static void hdf5_cleanup_file_runtime()
-{
- darshan_clear_record_refs(&(hdf5_file_runtime->hid_hash), 0);
- darshan_clear_record_refs(&(hdf5_file_runtime->rec_id_hash), 1);
-
- free(hdf5_file_runtime);
- hdf5_file_runtime = NULL;
-
- return;
-}
-
-static void hdf5_cleanup_dataset_runtime()
-{
- darshan_clear_record_refs(&(hdf5_dataset_runtime->hid_hash), 0);
- darshan_clear_record_refs(&(hdf5_dataset_runtime->rec_id_hash), 1);
-
- free(hdf5_dataset_runtime);
- hdf5_dataset_runtime = NULL;
-
- return;
-}
-
#ifdef HAVE_MPI
static void hdf5_file_record_reduction_op(void* inrec_v, void* inoutrec_v,
int *len, MPI_Datatype *datatype)
@@ -1620,7 +1600,7 @@ static void hdf5_dataset_mpi_redux(
}
#endif
-static void hdf5_file_shutdown(
+static void hdf5_file_output(
void **hdf5_buf,
int *hdf5_buf_sz)
{
@@ -1631,9 +1611,6 @@ static void hdf5_file_shutdown(
rec_count = hdf5_file_runtime->rec_count;
- /* shutdown internal structures used for instrumenting */
- hdf5_cleanup_file_runtime();
-
/* update output buffer size to account for shared dataset reduction */
*hdf5_buf_sz = rec_count * sizeof(struct darshan_hdf5_file);
@@ -1641,7 +1618,7 @@ static void hdf5_file_shutdown(
return;
}
-static void hdf5_dataset_shutdown(
+static void hdf5_dataset_output(
void **hdf5_buf,
int *hdf5_buf_sz)
{
@@ -1652,17 +1629,42 @@ static void hdf5_dataset_shutdown(
rec_count = hdf5_dataset_runtime->rec_count;
- /* perform any final transformations on MPIIO file records before
- * writing them out to log file
- */
+ /* update output buffer size to account for shared dataset reduction */
+ *hdf5_buf_sz = rec_count * sizeof(struct darshan_hdf5_dataset);
+
+ HDF5_UNLOCK();
+ return;
+}
+
+static void hdf5_file_cleanup()
+{
+ HDF5_LOCK();
+ assert(hdf5_file_runtime);
+
+ /* cleanup internal structures used for instrumenting */
+ darshan_clear_record_refs(&(hdf5_file_runtime->hid_hash), 0);
+ darshan_clear_record_refs(&(hdf5_file_runtime->rec_id_hash), 1);
+
+ free(hdf5_file_runtime);
+ hdf5_file_runtime = NULL;
+
+ HDF5_UNLOCK();
+ return;
+}
+
+static void hdf5_dataset_cleanup()
+{
+ HDF5_LOCK();
+ assert(hdf5_dataset_runtime);
+
+ /* cleanup internal structures used for instrumenting */
darshan_iter_record_refs(hdf5_dataset_runtime->rec_id_hash,
&hdf5_finalize_dataset_records, NULL);
+ darshan_clear_record_refs(&(hdf5_dataset_runtime->hid_hash), 0);
+ darshan_clear_record_refs(&(hdf5_dataset_runtime->rec_id_hash), 1);
- /* shutdown internal structures used for instrumenting */
- hdf5_cleanup_dataset_runtime();
-
- /* update output buffer size to account for shared dataset reduction */
- *hdf5_buf_sz = rec_count * sizeof(struct darshan_hdf5_dataset);
+ free(hdf5_dataset_runtime);
+ hdf5_dataset_runtime = NULL;
HDF5_UNLOCK();
return;
=====================================
darshan-runtime/lib/darshan-lustre.c
=====================================
@@ -40,8 +40,10 @@ static void lustre_mpi_redux(
void *lustre_buf, MPI_Comm mod_comm,
darshan_record_id *shared_recs, int shared_rec_count);
#endif
-static void lustre_shutdown(
+static void lustre_output(
void **lustre_buf, int *lustre_buf_sz);
+static void lustre_cleanup(
+ void);
struct lustre_runtime *lustre_runtime = NULL;
static pthread_mutex_t lustre_runtime_mutex = PTHREAD_RECURSIVE_MUTEX_INITIALIZER_NP;
@@ -186,7 +188,8 @@ static void lustre_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &lustre_mpi_redux,
#endif
- .mod_shutdown_func = &lustre_shutdown
+ .mod_output_func = &lustre_output,
+ .mod_cleanup_func = &lustre_cleanup
};
@@ -269,7 +272,7 @@ static void lustre_mpi_redux(
}
#endif
-static void lustre_shutdown(
+static void lustre_output(
void **lustre_buf,
int *lustre_buf_sz)
{
@@ -297,6 +300,15 @@ static void lustre_shutdown(
/* modify output buffer size to account for any shared records that were removed */
*lustre_buf_sz = lustre_runtime->record_buffer_size;
+ LUSTRE_UNLOCK();
+ return;
+}
+
+static void lustre_cleanup()
+{
+ LUSTRE_LOCK();
+ assert(lustre_runtime);
+
/* cleanup data structures */
darshan_clear_record_refs(&(lustre_runtime->record_id_hash), 1);
free(lustre_runtime);
=====================================
darshan-runtime/lib/darshan-mdhim.c
=====================================
@@ -85,17 +85,17 @@ static void mdhim_runtime_initialize(
void);
static struct mdhim_record_ref *mdhim_track_new_record(
darshan_record_id rec_id, int nr_servers, const char *name);
-static void mdhim_cleanup_runtime(
- void);
-/* forward declaration for MDHIM shutdown function needed to interface
+/* forward declaration for MDHIM functions needed to interface
* with darshan-core
*/
static void mdhim_mpi_redux(
void *mdhim_buf, MPI_Comm mod_comm,
darshan_record_id *shared_recs, int shared_rec_count);
-static void mdhim_shutdown(
+static void mdhim_output(
void **mdhim_buf, int *mdhim_buf_sz);
+static void mdhim_cleanup(
+ void);
/* mdhim_runtime is the global data structure encapsulating "MDHIM"
* module state */
@@ -310,7 +310,8 @@ static void mdhim_runtime_initialize()
int mdhim_buf_size;
darshan_module_funcs mod_funcs = {
.mod_redux_func = &mdhim_mpi_redux,
- .mod_shutdown_func = &mdhim_shutdown
+ .mod_output_func = &mdhim_output,
+ .mod_cleanup_func = &mdhim_cleanup
};
/* try and store a default number of records for this module */
@@ -399,18 +400,6 @@ static struct mdhim_record_ref *mdhim_track_new_record(
return(rec_ref);
}
-/* cleanup MDHIM module internal data structures */
-static void mdhim_cleanup_runtime()
-{
- /* iterate the hash of record references and free them */
- darshan_clear_record_refs(&(mdhim_runtime->rec_id_hash), 1);
-
- free(mdhim_runtime);
- mdhim_runtime = NULL;
-
- return;
-}
-
static void mdhim_record_reduction_op(void *infile_v, void *inoutfile_v,
int *len, MPI_Datatype *datatype)
{
@@ -477,10 +466,9 @@ static void mdhim_record_reduction_op(void *infile_v, void *inoutfile_v,
}
return;
}
-/***********************************************************************
- * shutdown function exported by the MDHIM module for coordinating with
- * darshan-core *
- ***********************************************************************/
+/*****************************************************************************
+ * functions exported by the MDHIM module for coordinating with darshan-core *
+ *****************************************************************************/
static void mdhim_mpi_redux(
void *mdhim_buf, MPI_Comm mod_comm,
@@ -578,10 +566,9 @@ static void mdhim_mpi_redux(
return;
}
-/* Pass output data for the MDHIM module back to darshan-core to log to
- * file, and shutdown/free internal data structures.
+/* Pass output data for the MDHIM module back to darshan-core to log to file
*/
-static void mdhim_shutdown(
+static void mdhim_output(
void **mdhim_buf,
int *mdhim_buf_sz)
{
@@ -590,8 +577,20 @@ static void mdhim_shutdown(
*mdhim_buf_sz = mdhim_runtime->rec_count * mdhim_runtime->record_size;
- /* shutdown internal structures used for instrumenting */
- mdhim_cleanup_runtime();
+ MDHIM_UNLOCK();
+ return;
+}
+
+static void mdhim_cleanup()
+{
+ MDHIM_LOCK();
+ assert(mdhim_runtime);
+
+ /* iterate the hash of record references and free them */
+ darshan_clear_record_refs(&(mdhim_runtime->rec_id_hash), 1);
+
+ free(mdhim_runtime);
+ mdhim_runtime = NULL;
MDHIM_UNLOCK();
return;
=====================================
darshan-runtime/lib/darshan-mpiio.c
=====================================
@@ -161,8 +161,6 @@ static struct mpiio_file_record_ref *mpiio_track_new_file_record(
darshan_record_id rec_id, const char *path);
static void mpiio_finalize_file_records(
void *rec_ref_p, void *user_ptr);
-static void mpiio_cleanup_runtime(
- void);
#ifdef HAVE_MPI
static void mpiio_record_reduction_op(
void* infile_v, void* inoutfile_v, int *len, MPI_Datatype *datatype);
@@ -173,8 +171,10 @@ static void mpiio_mpi_redux(
void *mpiio_buf, MPI_Comm mod_comm,
darshan_record_id *shared_recs, int shared_rec_count);
#endif
-static void mpiio_shutdown(
+static void mpiio_output(
void **mpiio_buf, int *mpiio_buf_sz);
+static void mpiio_cleanup(
+ void);
static struct mpiio_runtime *mpiio_runtime = NULL;
static pthread_mutex_t mpiio_runtime_mutex = PTHREAD_RECURSIVE_MUTEX_INITIALIZER_NP;
@@ -1160,7 +1160,8 @@ static void mpiio_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &mpiio_mpi_redux,
#endif
- .mod_shutdown_func = &mpiio_shutdown
+ .mod_output_func = &mpiio_output,
+ .mod_cleanup_func = &mpiio_cleanup
};
/* try and store the default number of records for this module */
@@ -1252,17 +1253,6 @@ static void mpiio_finalize_file_records(void *rec_ref_p, void *user_ptr)
return;
}
-static void mpiio_cleanup_runtime()
-{
- darshan_clear_record_refs(&(mpiio_runtime->fh_hash), 0);
- darshan_clear_record_refs(&(mpiio_runtime->rec_id_hash), 1);
-
- free(mpiio_runtime);
- mpiio_runtime = NULL;
-
- return;
-}
-
#ifdef HAVE_MPI
static void mpiio_record_reduction_op(void* infile_v, void* inoutfile_v,
int *len, MPI_Datatype *datatype)
@@ -1534,7 +1524,7 @@ void darshan_mpiio_shutdown_bench_setup(int test_case)
intptr_t j;
if(mpiio_runtime)
- mpiio_cleanup_runtime();
+ mpiio_cleanup();
mpiio_runtime_initialize();
@@ -1612,7 +1602,7 @@ void darshan_mpiio_shutdown_bench_setup(int test_case)
}
/********************************************************************************
- * shutdown function exported by this module for coordinating with darshan-core *
+ * functions exported by this module for coordinating with darshan-core *
********************************************************************************/
#ifdef HAVE_MPI
@@ -1731,7 +1721,7 @@ static void mpiio_mpi_redux(
}
#endif
-static void mpiio_shutdown(
+static void mpiio_output(
void **mpiio_buf,
int *mpiio_buf_sz)
{
@@ -1742,16 +1732,25 @@ static void mpiio_shutdown(
mpiio_rec_count = mpiio_runtime->file_rec_count;
- /* perform any final transformations on MPIIO file records before
- * writing them out to log file
- */
+ *mpiio_buf_sz = mpiio_rec_count * sizeof(struct darshan_mpiio_file);
+
+ MPIIO_UNLOCK();
+ return;
+}
+
+static void mpiio_cleanup()
+{
+ MPIIO_LOCK();
+ assert(mpiio_runtime);
+
+ /* cleanup internal structures used for instrumenting */
darshan_iter_record_refs(mpiio_runtime->rec_id_hash,
&mpiio_finalize_file_records, NULL);
+ darshan_clear_record_refs(&(mpiio_runtime->fh_hash), 0);
+ darshan_clear_record_refs(&(mpiio_runtime->rec_id_hash), 1);
- /* shutdown internal structures used for instrumenting */
- mpiio_cleanup_runtime();
-
- *mpiio_buf_sz = mpiio_rec_count * sizeof(struct darshan_mpiio_file);
+ free(mpiio_runtime);
+ mpiio_runtime = NULL;
MPIIO_UNLOCK();
return;
=====================================
darshan-runtime/lib/darshan-null.c
=====================================
@@ -81,13 +81,12 @@ static void null_runtime_initialize(
void);
static struct null_record_ref *null_track_new_record(
darshan_record_id rec_id, const char *name);
-static void null_cleanup_runtime(
- void);
-/* forward declaration for NULL shutdown function needed to interface
+/* forward declaration for NULL output/cleanup functions needed to interface
* with darshan-core
*/
-static void null_shutdown(void **null_buf, int *null_buf_sz);
+static void null_output(void **null_buf, int *null_buf_sz);
+static void null_cleanup(void);
/* null_runtime is the global data structure encapsulating "NULL" module state */
static struct null_runtime *null_runtime = NULL;
@@ -210,7 +209,8 @@ static void null_runtime_initialize()
*/
.mod_redux_func = NULL,
#endif
- .mod_shutdown_func = &null_shutdown
+ .mod_output_func = &null_output,
+ .mod_cleanup_func = &null_cleanup
};
/* try and store a default number of records for this module */
@@ -298,26 +298,13 @@ static struct null_record_ref *null_track_new_record(
return(rec_ref);
}
-/* cleanup NULL module internal data structures */
-static void null_cleanup_runtime()
-{
- /* iterate the hash of record references and free them */
- darshan_clear_record_refs(&(null_runtime->rec_id_hash), 1);
-
- free(null_runtime);
- null_runtime = NULL;
-
- return;
-}
-
/**************************************************************************************
* shutdown function exported by the "NULL" module for coordinating with darshan-core *
**************************************************************************************/
-/* Pass output data for the "NULL" module back to darshan-core to log to file,
- * and shutdown/free internal data structures.
+/* Pass output data for the "NULL" module back to darshan-core to log to file
*/
-static void null_shutdown(
+static void null_output(
void **null_buf,
int *null_buf_sz)
{
@@ -332,8 +319,22 @@ static void null_shutdown(
*/
*null_buf_sz = null_runtime->rec_count * sizeof(struct darshan_null_record);
+ NULL_UNLOCK();
+ return;
+}
+
+/* Shutdown/free internal data structures
+ */
+static void null_cleanup()
+{
/* shutdown internal structures used for instrumenting */
- null_cleanup_runtime();
+ NULL_LOCK();
+
+ /* iterate the hash of record references and free them */
+ darshan_clear_record_refs(&(null_runtime->rec_id_hash), 1);
+
+ free(null_runtime);
+ null_runtime = NULL;
NULL_UNLOCK();
return;
=====================================
darshan-runtime/lib/darshan-pnetcdf.c
=====================================
@@ -46,8 +46,6 @@ static void pnetcdf_runtime_initialize(
void);
static struct pnetcdf_file_record_ref *pnetcdf_track_new_file_record(
darshan_record_id rec_id, const char *path);
-static void pnetcdf_cleanup_runtime(
- void);
#ifdef HAVE_MPI
static void pnetcdf_record_reduction_op(
void* infile_v, void* inoutfile_v, int *len, MPI_Datatype *datatype);
@@ -55,8 +53,10 @@ static void pnetcdf_mpi_redux(
void *pnetcdf_buf, MPI_Comm mod_comm,
darshan_record_id *shared_recs, int shared_rec_count);
#endif
-static void pnetcdf_shutdown(
+static void pnetcdf_output(
void **pnetcdf_buf, int *pnetcdf_buf_sz);
+static void pnetcdf_cleanup(
+ void);
static struct pnetcdf_runtime *pnetcdf_runtime = NULL;
static pthread_mutex_t pnetcdf_runtime_mutex = PTHREAD_RECURSIVE_MUTEX_INITIALIZER_NP;
@@ -217,7 +217,8 @@ static void pnetcdf_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &pnetcdf_mpi_redux,
#endif
- .mod_shutdown_func = &pnetcdf_shutdown
+ .mod_output_func = &pnetcdf_output,
+ .mod_cleanup_func = &pnetcdf_cleanup
};
/* try and store the default number of records for this module */
@@ -297,18 +298,6 @@ static struct pnetcdf_file_record_ref *pnetcdf_track_new_file_record(
return(rec_ref);
}
-static void pnetcdf_cleanup_runtime()
-{
- darshan_clear_record_refs(&(pnetcdf_runtime->ncid_hash), 0);
- darshan_clear_record_refs(&(pnetcdf_runtime->rec_id_hash), 1);
-
- free(pnetcdf_runtime);
- pnetcdf_runtime = NULL;
-
- return;
-}
-
-#ifdef HAVE_MPI
static void pnetcdf_record_reduction_op(void* infile_v, void* inoutfile_v,
int *len, MPI_Datatype *datatype)
{
@@ -450,7 +439,7 @@ static void pnetcdf_mpi_redux(
}
#endif
-static void pnetcdf_shutdown(
+static void pnetcdf_output(
void **pnetcdf_buf,
int *pnetcdf_buf_sz)
{
@@ -461,9 +450,6 @@ static void pnetcdf_shutdown(
pnetcdf_rec_count = pnetcdf_runtime->file_rec_count;
- /* shutdown internal structures used for instrumenting */
- pnetcdf_cleanup_runtime();
-
/* update output buffer size to account for shared file reduction */
*pnetcdf_buf_sz = pnetcdf_rec_count * sizeof(struct darshan_pnetcdf_file);
@@ -471,6 +457,22 @@ static void pnetcdf_shutdown(
return;
}
+static void pnetcdf_cleanup()
+{
+ PNETCDF_LOCK();
+ assert(pnetcdf_runtime);
+
+ /* cleanup internal structures used for instrumenting */
+ darshan_clear_record_refs(&(pnetcdf_runtime->ncid_hash), 0);
+ darshan_clear_record_refs(&(pnetcdf_runtime->rec_id_hash), 1);
+
+ free(pnetcdf_runtime);
+ pnetcdf_runtime = NULL;
+
+ PNETCDF_UNLOCK();
+ return;
+}
+
/*
* Local variables:
* c-indent-level: 4
=====================================
darshan-runtime/lib/darshan-posix.c
=====================================
@@ -154,8 +154,6 @@ static struct posix_aio_tracker* posix_aio_tracker_del(
int fd, void *aiocbp);
static void posix_finalize_file_records(
void *rec_ref_p, void *user_ptr);
-static void posix_cleanup_runtime(
- void);
#ifdef HAVE_MPI
static void posix_record_reduction_op(
void* infile_v, void* inoutfile_v, int *len, MPI_Datatype *datatype);
@@ -166,8 +164,10 @@ static void posix_mpi_redux(
void *posix_buf, MPI_Comm mod_comm,
darshan_record_id *shared_recs, int shared_rec_count);
#endif
-static void posix_shutdown(
+static void posix_output(
void **posix_buf, int *posix_buf_sz);
+static void posix_cleanup(
+ void);
/* extern function def for querying record name from a STDIO stream */
extern char *darshan_stdio_lookup_record_name(FILE *stream);
@@ -1490,7 +1490,8 @@ static void posix_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &posix_mpi_redux,
#endif
- .mod_shutdown_func = &posix_shutdown
+ .mod_output_func = &posix_output,
+ .mod_cleanup_func = &posix_cleanup
};
/* try and store a default number of records for this module */
@@ -1639,17 +1640,6 @@ static void posix_finalize_file_records(void *rec_ref_p, void *user_ptr)
return;
}
-static void posix_cleanup_runtime()
-{
- darshan_clear_record_refs(&(posix_runtime->fd_hash), 0);
- darshan_clear_record_refs(&(posix_runtime->rec_id_hash), 1);
-
- free(posix_runtime);
- posix_runtime = NULL;
-
- return;
-}
-
#ifdef HAVE_MPI
static void posix_record_reduction_op(void* infile_v, void* inoutfile_v,
int *len, MPI_Datatype *datatype)
@@ -2009,7 +1999,7 @@ void darshan_posix_shutdown_bench_setup(int test_case)
int i;
if(posix_runtime)
- posix_cleanup_runtime();
+ posix_cleanup();
posix_runtime_initialize();
@@ -2195,7 +2185,7 @@ static void posix_mpi_redux(
}
#endif
-static void posix_shutdown(
+static void posix_output(
void **posix_buf,
int *posix_buf_sz)
{
@@ -2206,17 +2196,26 @@ static void posix_shutdown(
posix_rec_count = posix_runtime->file_rec_count;
- /* perform any final transformations on POSIX file records before
- * writing them out to log file
- */
- darshan_iter_record_refs(posix_runtime->rec_id_hash,
- &posix_finalize_file_records, NULL);
+ /* update output buffer size to account for shared file reduction */
+ *posix_buf_sz = posix_rec_count * sizeof(struct darshan_posix_file);
+
+ POSIX_UNLOCK();
+ return;
+}
+
+static void posix_cleanup()
+{
+ POSIX_LOCK();
+ assert(posix_runtime);
/* shutdown internal structures used for instrumenting */
- posix_cleanup_runtime();
+ darshan_iter_record_refs(posix_runtime->rec_id_hash,
+ &posix_finalize_file_records, NULL);
+ darshan_clear_record_refs(&(posix_runtime->fd_hash), 0);
+ darshan_clear_record_refs(&(posix_runtime->rec_id_hash), 1);
- /* update output buffer size to account for shared file reduction */
- *posix_buf_sz = posix_rec_count * sizeof(struct darshan_posix_file);
+ free(posix_runtime);
+ posix_runtime = NULL;
POSIX_UNLOCK();
return;
=====================================
darshan-runtime/lib/darshan-stdio.c
=====================================
@@ -152,8 +152,6 @@ static void stdio_runtime_initialize(
void);
static struct stdio_file_record_ref *stdio_track_new_file_record(
darshan_record_id rec_id, const char *path);
-static void stdio_cleanup_runtime(
- void);
#ifdef HAVE_MPI
static void stdio_record_reduction_op(void* infile_v, void* inoutfile_v,
int *len, MPI_Datatype *datatype);
@@ -164,8 +162,10 @@ static void stdio_mpi_redux(
void *stdio_buf, MPI_Comm mod_comm,
darshan_record_id *shared_recs, int shared_rec_count);
#endif
-static void stdio_shutdown(
+static void stdio_output(
void **stdio_buf, int *stdio_buf_sz);
+static void stdio_cleanup(
+ void);
/* extern function def for querying record name from a POSIX fd */
extern char *darshan_posix_lookup_record_name(int fd);
@@ -1004,7 +1004,8 @@ static void stdio_runtime_initialize()
#ifdef HAVE_MPI
.mod_redux_func = &stdio_mpi_redux,
#endif
- .mod_shutdown_func = &stdio_shutdown
+ .mod_output_func = &stdio_output,
+ .mod_cleanup_func = &stdio_cleanup
};
/* try to store default number of records for this module */
@@ -1089,17 +1090,6 @@ static struct stdio_file_record_ref *stdio_track_new_file_record(
return(rec_ref);
}
-static void stdio_cleanup_runtime()
-{
- darshan_clear_record_refs(&(stdio_runtime->stream_hash), 0);
- darshan_clear_record_refs(&(stdio_runtime->rec_id_hash), 1);
-
- free(stdio_runtime);
- stdio_runtime = NULL;
-
- return;
-}
-
#ifdef HAVE_MPI
static void stdio_record_reduction_op(void* infile_v, void* inoutfile_v,
int *len, MPI_Datatype *datatype)
@@ -1429,7 +1419,7 @@ static void stdio_mpi_redux(
}
#endif
-static void stdio_shutdown(
+static void stdio_output(
void **stdio_buf,
int *stdio_buf_sz)
{
@@ -1474,14 +1464,26 @@ static void stdio_shutdown(
}
}
- /* shutdown internal structures used for instrumenting */
- stdio_cleanup_runtime();
-
/* update output buffer size to account for shared file reduction */
*stdio_buf_sz = stdio_rec_count * sizeof(struct darshan_stdio_file);
STDIO_UNLOCK();
-
+ return;
+}
+
+static void stdio_cleanup()
+{
+ STDIO_LOCK();
+ assert(stdio_runtime);
+
+ /* shutdown internal structures used for instrumenting */
+ darshan_clear_record_refs(&(stdio_runtime->stream_hash), 0);
+ darshan_clear_record_refs(&(stdio_runtime->rec_id_hash), 1);
+
+ free(stdio_runtime);
+ stdio_runtime = NULL;
+
+ STDIO_UNLOCK();
return;
}
View it on GitLab: https://xgitlab.cels.anl.gov/darshan/darshan/-/commit/2b1b38f28cc594eee2efd2d5d703e8568235bf4d
--
View it on GitLab: https://xgitlab.cels.anl.gov/darshan/darshan/-/commit/2b1b38f28cc594eee2efd2d5d703e8568235bf4d
You're receiving this email because of your account on xgitlab.cels.anl.gov.
-------------- next part --------------
An HTML attachment was scrubbed...
URL: <http://lists.mcs.anl.gov/pipermail/darshan-commits/attachments/20210204/d72dc9b2/attachment-0001.html>
More information about the Darshan-commits
mailing list