[Darshan-commits] [Git][darshan/darshan][dev-modular] 2 commits: darshan-runtime commits for configurable mem
Shane Snyder
xgitlab at cels.anl.gov
Mon Nov 2 13:32:22 CST 2015
Shane Snyder pushed to branch dev-modular at darshan / darshan
Commits:
98c93e0f by Shane Snyder at 2015-11-02T12:34:38Z
darshan-runtime commits for configurable mem
user can now give a config option or use a runtime environment
variable to control darshan's per module memory requirements.
- - - - -
40d1dc03 by Shane Snyder at 2015-11-02T13:31:25Z
darshan-util updates to support partial logs
- - - - -
15 changed files:
- darshan-log-format.h
- darshan-runtime/configure
- darshan-runtime/configure.in
- darshan-runtime/darshan-core.h
- darshan-runtime/darshan-runtime-config.h.in
- darshan-runtime/darshan.h
- darshan-runtime/lib/darshan-bgq.c
- darshan-runtime/lib/darshan-core.c
- darshan-runtime/lib/darshan-hdf5.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-util/darshan-job-summary/bin/darshan-job-summary.pl.in
- darshan-util/darshan-parser.c
Changes:
=====================================
darshan-log-format.h
=====================================
--- a/darshan-log-format.h
+++ b/darshan-log-format.h
@@ -69,6 +69,11 @@ static char * const darshan_module_names[] =
};
#undef X
+/* simple macros for accessing module flag bitfields */
+#define DARSHAN_MOD_FLAG_SET(flags, id) flags = (flags | (1 << id))
+#define DARSHAN_MOD_FLAG_UNSET(flags, id) flags = (flags & ~(1 << id))
+#define DARSHAN_MOD_FLAG_ISSET(flags, id) (flags & (1 << id))
+
/* compression method used on darshan log file */
enum darshan_comp_type
{
@@ -97,7 +102,7 @@ struct darshan_header
char version_string[8];
int64_t magic_nr;
unsigned char comp_type;
- unsigned char partial_flag;
+ uint32_t partial_flag;
struct darshan_log_map rec_map;
struct darshan_log_map mod_map[DARSHAN_MAX_MODS];
};
=====================================
darshan-runtime/configure
=====================================
--- a/darshan-runtime/configure
+++ b/darshan-runtime/configure
@@ -693,7 +693,7 @@ with_log_hints
with_log_path
with_jobid_env
enable_bgq_mod
-with_max_records
+with_mod_mem
'
ac_precious_vars='build_alias
host_alias
@@ -1331,7 +1331,7 @@ Optional Packages:
--with-jobid-env=<name> Name of environment variable that stores the jobid
(specify "NONE" if no appropriate environment variable is available:
Darshan will use rank 0's pid instead)
- --with-max-records=<num> Maximum records for Darshan to track at runtime
+ --with-mod-mem=<num> Maximum amount of memory (in MiB) for each Darshan module
Some influential environment variables:
CC C compiler command
@@ -4281,14 +4281,14 @@ rm -f core conftest.err conftest.$ac_objext conftest.$ac_ext
fi
-# Check whether --with-max-records was given.
-if test "${with_max_records+set}" = set; then :
- withval=$with_max_records; if test x$withval = xyes; then
- as_fn_error $? "--with-max-records must be given a number" "$LINENO" 5
+# Check whether --with-mod-mem was given.
+if test "${with_mod_mem+set}" = set; then :
+ withval=$with_mod_mem; if test x$withval = xyes; then
+ as_fn_error $? "--with-mod-mem must be given a number" "$LINENO" 5
else
cat >>confdefs.h <<_ACEOF
-#define __DARSHAN_MAX_RECORDS ${withval}
+#define __DARSHAN_MOD_MEM_MAX ${withval}
_ACEOF
fi
=====================================
darshan-runtime/configure.in
=====================================
--- a/darshan-runtime/configure.in
+++ b/darshan-runtime/configure.in
@@ -300,12 +300,12 @@ if test x$enable_bgq_mod != xno; then
[]))
fi
-AC_ARG_WITH(max-records,
-[ --with-max-records=<num> Maximum records for Darshan to track at runtime],
+AC_ARG_WITH(mod-mem,
+[ --with-mod-mem=<num> Maximum amount of memory (in MiB) for each Darshan module],
if test x$withval = xyes; then
- AC_MSG_ERROR(--with-max-records must be given a number)
+ AC_MSG_ERROR(--with-mod-mem must be given a number)
else
- AC_DEFINE_UNQUOTED(__DARSHAN_MAX_RECORDS, ${withval}, Maximum Darshan records to track at runtime)
+ AC_DEFINE_UNQUOTED(__DARSHAN_MOD_MEM_MAX, ${withval}, Maximum memory (in MiB) for each Darshan module)
fi
)
=====================================
darshan-runtime/darshan-core.h
=====================================
--- a/darshan-runtime/darshan-core.h
+++ b/darshan-runtime/darshan-core.h
@@ -26,18 +26,18 @@
/* Environment variable to override __DARSHAN_MEM_ALIGNMENT */
#define DARSHAN_MEM_ALIGNMENT_OVERRIDE "DARSHAN_MEMALIGN"
-#ifdef __DARSHAN_MAX_RECORDS
-#define DARSHAN_CORE_MAX_RECORDS __DARSHAN_MAX_RECORDS
+/* Environment variable to override memory per module */
+#define DARSHAN_MOD_MEM_OVERRIDE "DARSHAN_MODMEM"
+
+/* Maximum amount of memory per instrumentation module in MiB */
+#ifdef __DARSHAN_MOD_MEM_MAX
+#define DARSHAN_MOD_MEM_MAX (__DARSHAN_MOD_MEM_MAX * 1024 * 1024)
#else
-#define DARSHAN_CORE_MAX_RECORDS 2048
+#define DARSHAN_MOD_MEM_MAX (2 * 1024 * 1024) /* 2 MiB default */
#endif
-/* TODO: revisit this default size if we change memory per module */
-#define DARSHAN_CORE_COMP_BUF_SIZE (2 * 1024 * 1024)
-
-#define DARSHAN_CORE_MOD_SET(flags, id) (flags | (1 << id))
-#define DARSHAN_CORE_MOD_UNSET(flags, id) (flags & ~(1 << id))
-#define DARSHAN_CORE_MOD_ISSET(flags, id) (flags & (1 << id))
+/* Default runtime compression buffer size */
+#define DARSHAN_COMP_BUF_SIZE DARSHAN_MOD_MEM_MAX
/* in memory structure to keep up with job level data */
struct darshan_core_runtime
@@ -48,7 +48,7 @@ struct darshan_core_runtime
struct darshan_core_record_ref *rec_hash;
int rec_count;
struct darshan_core_module* mod_array[DARSHAN_MAX_MODS];
- char comp_buf[DARSHAN_CORE_COMP_BUF_SIZE];
+ char comp_buf[DARSHAN_COMP_BUF_SIZE];
double wtime_offset;
char *trailing_data;
};
=====================================
darshan-runtime/darshan-runtime-config.h.in
=====================================
--- a/darshan-runtime/darshan-runtime-config.h.in
+++ b/darshan-runtime/darshan-runtime-config.h.in
@@ -102,11 +102,11 @@
/* Location to store log files at run time */
#undef __DARSHAN_LOG_PATH
-/* Maximum Darshan records to track at runtime */
-#undef __DARSHAN_MAX_RECORDS
-
/* Memory alignment in bytes */
#undef __DARSHAN_MEM_ALIGNMENT
+/* Maximum memory (in MiB) for each Darshan module */
+#undef __DARSHAN_MOD_MEM_MAX
+
/* Generalized request type for MPI-IO */
#undef __D_MPI_REQUEST
=====================================
darshan-runtime/darshan.h
=====================================
--- a/darshan-runtime/darshan.h
+++ b/darshan-runtime/darshan.h
@@ -118,18 +118,21 @@ void darshan_core_unregister_module(
* Register the Darshan record given by 'name' with the darshan-core
* runtime, allowing it to be properly tracked and (potentially)
* correlated with records from other modules. 'len' is the size of
- * the name pointer (string length for string names), 'printable_flag'
- * indicates whether the name is a string, and 'mod_id' is the identifier
- * of the calling module. 'rec_id' is an output pointer storing the
- * correspoing Darshan record identifier and 'file_alignment' is an
- * output pointer storing the file system alignment value for the given
- * record.
+ * the name pointer (string length for string names), and 'printable_flag'
+ * indicates whether the name is a string. 'mod_limit_flag' is set if
+ * the calling module is out of memory (to prevent darshan-core from
+ * creating new records and to just search existing records) and 'mod_id'
+ * is the identifier of the calling module. 'rec_id' is an output pointer
+ * storing the correspoing Darshan record identifier and 'file_alignment'
+ * is an output pointer storing the file system alignment value for the
+ * given record.
*/
void darshan_core_register_record(
void *name,
int len,
- int printable_flag,
darshan_module_id mod_id,
+ int printable_flag,
+ int mod_limit_flag,
darshan_record_id *rec_id,
int *file_alignment);
=====================================
darshan-runtime/lib/darshan-bgq.c
=====================================
--- a/darshan-runtime/lib/darshan-bgq.c
+++ b/darshan-runtime/lib/darshan-bgq.c
@@ -156,8 +156,9 @@ void bgq_runtime_initialize()
darshan_core_register_record(
recname,
strlen(recname),
- 1,
DARSHAN_BGQ_MOD,
+ 1,
+ 0,
&bgq_runtime->record.f_id,
&bgq_runtime->record.alignment);
=====================================
darshan-runtime/lib/darshan-core.c
=====================================
--- a/darshan-runtime/lib/darshan-core.c
+++ b/darshan-runtime/lib/darshan-core.c
@@ -99,7 +99,8 @@ static char* darshan_get_exe_and_mounts(
static void darshan_block_size_from_path(
const char *path, int *block_size);
static void darshan_get_shared_records(
- struct darshan_core_runtime *core, darshan_record_id *shared_recs);
+ struct darshan_core_runtime *core, darshan_record_id **shared_recs,
+ int *shared_rec_cnt);
static int darshan_log_open_all(
char *logfile_name, MPI_File *log_fh);
static int darshan_deflate_buffer(
@@ -251,7 +252,9 @@ void darshan_core_shutdown()
int64_t last_end_time;
int local_mod_use[DARSHAN_MAX_MODS] = {0};
int global_mod_use_count[DARSHAN_MAX_MODS] = {0};
- darshan_record_id shared_recs[DARSHAN_CORE_MAX_RECORDS] = {0};
+ darshan_record_id *shared_recs;
+ darshan_record_id *mod_shared_recs;
+ int shared_rec_cnt = 0;
double start_log_time;
double open1, open2;
double job1, job2;
@@ -261,7 +264,7 @@ void darshan_core_shutdown()
double header1, header2;
double tm_end;
uint64_t gz_fp = 0;
- unsigned char tmp_partial_flag;
+ uint32_t tmp_partial_flag;
MPI_File log_fh;
MPI_Status status;
@@ -368,7 +371,7 @@ void darshan_core_shutdown()
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);
+ darshan_get_shared_records(final_core, &shared_recs, &shared_rec_cnt);
if(internal_timing_flag)
open1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
@@ -463,6 +466,9 @@ void darshan_core_shutdown()
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
@@ -475,7 +481,6 @@ void darshan_core_shutdown()
{
struct darshan_core_module* this_mod = final_core->mod_array[i];
struct darshan_core_record_ref *ref = NULL;
- darshan_record_id mod_shared_recs[DARSHAN_CORE_MAX_RECORDS];
int mod_shared_rec_cnt = 0;
void* mod_buf = NULL;
int mod_buf_sz = 0;
@@ -495,13 +500,13 @@ void darshan_core_shutdown()
mod1[i] = DARSHAN_MPI_CALL(PMPI_Wtime)();
/* set the shared file list for this module */
- memset(mod_shared_recs, 0, DARSHAN_CORE_MAX_RECORDS * sizeof(darshan_record_id));
- for(j = 0; j < DARSHAN_CORE_MAX_RECORDS && shared_recs[j] != 0; j++)
+ 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_CORE_MOD_ISSET(ref->global_mod_flags, i))
+ if(DARSHAN_MOD_FLAG_ISSET(ref->global_mod_flags, i))
{
mod_shared_recs[mod_shared_rec_cnt++] = shared_recs[j];
}
@@ -551,11 +556,11 @@ void darshan_core_shutdown()
}
/* run a reduction to determine if any application processes had to set the
- * partial flag. this happens when a process has tracked too many records
- * at once and cannot track new records
+ * partial flag for any modules. this happens when a module exhausts its memory
+ * and does not track every possible record
*/
DARSHAN_MPI_CALL(PMPI_Reduce)(&(final_core->log_header.partial_flag),
- &tmp_partial_flag, 1, MPI_UNSIGNED_CHAR, MPI_MAX, 0, MPI_COMM_WORLD);
+ &tmp_partial_flag, 1, MPI_UINT32_T, MPI_BOR, 0, MPI_COMM_WORLD);
if(internal_timing_flag)
header1 = DARSHAN_MPI_CALL(PMPI_Wtime)();
@@ -1085,32 +1090,45 @@ static void darshan_block_size_from_path(const char *path, int *block_size)
}
static void darshan_get_shared_records(struct darshan_core_runtime *core,
- darshan_record_id *shared_recs)
+ darshan_record_id **shared_recs, int *shared_rec_cnt)
{
- int i;
- int ndx;
+ int i, j;
+ int tmp_cnt = core->rec_count;
struct darshan_core_record_ref *tmp, *ref;
- darshan_record_id id_array[DARSHAN_CORE_MAX_RECORDS] = {0};
- uint64_t mod_flags[DARSHAN_CORE_MAX_RECORDS] = {0};
- uint64_t global_mod_flags[DARSHAN_CORE_MAX_RECORDS] = {0};
+ darshan_record_id *id_array;
+ uint64_t *mod_flags;
+ uint64_t *global_mod_flags;
+
+ /* broadcast root's number of records to all other processes */
+ DARSHAN_MPI_CALL(PMPI_Bcast)(&tmp_cnt, 1, MPI_INT, 0, MPI_COMM_WORLD);
+
+ /* use root record count to allocate data structures */
+ id_array = malloc(tmp_cnt * sizeof(darshan_record_id));
+ mod_flags = malloc(tmp_cnt * sizeof(uint64_t));
+ global_mod_flags = malloc(tmp_cnt * sizeof(uint64_t));
+ *shared_recs = malloc(tmp_cnt * sizeof(darshan_record_id));
+ assert(id_array && mod_flags && global_mod_flags && *shared_recs);
+
+ memset(mod_flags, 0, tmp_cnt * sizeof(uint64_t));
+ memset(global_mod_flags, 0, tmp_cnt * sizeof(uint64_t));
+ memset(*shared_recs, 0, tmp_cnt * sizeof(darshan_record_id));
/* first, determine list of records root process has opened */
if(my_rank == 0)
{
- ndx = 0;
+ i = 0;
HASH_ITER(hlink, core->rec_hash, ref, tmp)
{
- id_array[ndx++] = ref->rec.id;
+ id_array[i++] = ref->rec.id;
}
}
/* broadcast root's list of records to all other processes */
- DARSHAN_MPI_CALL(PMPI_Bcast)(id_array,
- (DARSHAN_CORE_MAX_RECORDS * sizeof(darshan_record_id)),
+ DARSHAN_MPI_CALL(PMPI_Bcast)(id_array, (tmp_cnt * sizeof(darshan_record_id)),
MPI_BYTE, 0, MPI_COMM_WORLD);
/* everyone looks to see if they opened the same records as root */
- for(i=0; (i<DARSHAN_CORE_MAX_RECORDS && id_array[i] != 0); i++)
+ for(i=0; i<tmp_cnt; i++)
{
HASH_FIND(hlink, core->rec_hash, &id_array[i], sizeof(darshan_record_id), ref);
if(ref)
@@ -1123,15 +1141,15 @@ static void darshan_get_shared_records(struct darshan_core_runtime *core,
/* now allreduce so everyone agrees which files are shared and
* which modules accessed them collectively
*/
- DARSHAN_MPI_CALL(PMPI_Allreduce)(mod_flags, global_mod_flags,
- DARSHAN_CORE_MAX_RECORDS, MPI_UINT64_T, MPI_BAND, MPI_COMM_WORLD);
+ DARSHAN_MPI_CALL(PMPI_Allreduce)(mod_flags, global_mod_flags, tmp_cnt,
+ MPI_UINT64_T, MPI_BAND, MPI_COMM_WORLD);
- ndx = 0;
- for(i=0; (i<DARSHAN_CORE_MAX_RECORDS && id_array[i] != 0); i++)
+ j = 0;
+ for(i=0; i<tmp_cnt; i++)
{
if(global_mod_flags[i] != 0)
{
- shared_recs[ndx++] = id_array[i];
+ (*shared_recs)[j++] = id_array[i];
/* set global_mod_flags so we know which modules collectively
* accessed this module. we need this info to support shared
@@ -1142,6 +1160,7 @@ static void darshan_get_shared_records(struct darshan_core_runtime *core,
ref->global_mod_flags = global_mod_flags[i];
}
}
+ *shared_rec_cnt = j;
return;
}
@@ -1246,7 +1265,7 @@ static int darshan_deflate_buffer(void **pointers, int *lengths, int count,
}
tmp_stream.next_out = (unsigned char *)comp_buf;
- tmp_stream.avail_out = DARSHAN_CORE_COMP_BUF_SIZE;
+ tmp_stream.avail_out = DARSHAN_COMP_BUF_SIZE;
/* loop over the input pointers */
for(i = 0; i < count; i++)
@@ -1307,9 +1326,9 @@ static int darshan_log_write_record_hash(MPI_File log_fh, struct darshan_core_ru
char *hash_buf;
char *hash_buf_off;
- /* allocate a buffer to store at most 64 bytes for each of a max number of records */
+ /* 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 = DARSHAN_CORE_MAX_RECORDS * 64;
+ hash_buf_sz = core->rec_count * 64;
hash_buf = malloc(hash_buf_sz);
if(!hash_buf)
{
@@ -1478,7 +1497,10 @@ void darshan_core_register_module(
int *mod_mem_limit,
int *sys_mem_alignment)
{
+ int ret;
+ int tmpval;
struct darshan_core_module* mod;
+ char *mod_mem_str = NULL;
*mod_mem_limit = 0;
if(!darshan_core || (mod_id >= DARSHAN_MAX_MODS))
@@ -1514,9 +1536,21 @@ void darshan_core_register_module(
/* get the calling process's rank */
DARSHAN_MPI_CALL(PMPI_Comm_rank)(MPI_COMM_WORLD, my_rank);
- /* TODO: something smarter than just 2 MiB per module */
- *mod_mem_limit = 2 * 1024 * 1024;
-
+ /* set the maximum amount of memory this module can use */
+ mod_mem_str = getenv(DARSHAN_MOD_MEM_OVERRIDE);
+ if(mod_mem_str)
+ {
+ ret = sscanf(mod_mem_str, "%d", &tmpval);
+ /* silently ignore if the env variable is set poorly */
+ if(ret == 1 && tmpval > 0)
+ *mod_mem_limit = (tmpval * 1024 * 1024); /* convert to MiB */
+ else
+ *mod_mem_limit = DARSHAN_MOD_MEM_MAX;
+ }
+ else
+ {
+ *mod_mem_limit = DARSHAN_MOD_MEM_MAX;
+ }
DARSHAN_CORE_UNLOCK();
return;
@@ -1550,8 +1584,9 @@ void darshan_core_unregister_module(
void darshan_core_register_record(
void *name,
int len,
- int printable_flag,
darshan_module_id mod_id,
+ int printable_flag,
+ int mod_limit_flag,
darshan_record_id *rec_id,
int *file_alignment)
{
@@ -1573,16 +1608,14 @@ void darshan_core_register_record(
HASH_FIND(hlink, darshan_core->rec_hash, &tmp_rec_id, sizeof(darshan_record_id), ref);
if(!ref)
{
- /* record not found -- add it to the hash if we aren't already tracking the
- * maximum number of records
+ /* record not found -- add it to the hash if this module has not already used
+ * all of its memory
*/
- if(darshan_core->rec_count >= DARSHAN_CORE_MAX_RECORDS)
+ if(mod_limit_flag)
{
- /* if we are already tracking the max records, set a flag to indicate
- * that this log file has partial results
- */
- darshan_core->log_header.partial_flag = 1;
+ /* if this module is OOM, set a flag in the header to indicate this */
+ DARSHAN_MOD_FLAG_SET(darshan_core->log_header.partial_flag, mod_id);
DARSHAN_CORE_UNLOCK();
return;
}
@@ -1600,7 +1633,7 @@ void darshan_core_register_record(
darshan_core->rec_count++;
}
}
- ref->mod_flags = DARSHAN_CORE_MOD_SET(ref->mod_flags, mod_id);
+ DARSHAN_MOD_FLAG_SET(ref->mod_flags, mod_id);
DARSHAN_CORE_UNLOCK();
if(file_alignment)
@@ -1625,7 +1658,7 @@ void darshan_core_unregister_record(
assert(ref);
/* disassociate this module from the given record id */
- ref->mod_flags = DARSHAN_CORE_MOD_UNSET(ref->mod_flags, mod_id);
+ DARSHAN_MOD_FLAG_UNSET(ref->mod_flags, mod_id);
if(!(ref->mod_flags))
{
/* if no other modules are associated with this rec, delete it */
=====================================
darshan-runtime/lib/darshan-hdf5.c
=====================================
--- a/darshan-runtime/lib/darshan-hdf5.c
+++ b/darshan-runtime/lib/darshan-hdf5.c
@@ -256,6 +256,7 @@ static struct hdf5_file_runtime* hdf5_file_by_name(const char *name)
struct hdf5_file_runtime *file = NULL;
char *newname = NULL;
darshan_record_id file_id;
+ int limit_flag;
if(!hdf5_runtime || instrumentation_disabled)
return(NULL);
@@ -264,12 +265,15 @@ static struct hdf5_file_runtime* hdf5_file_by_name(const char *name)
if(!newname)
newname = (char*)name;
+ limit_flag = (hdf5_runtime->file_array_ndx >= hdf5_runtime->file_array_size);
+
/* get a unique id for this file from darshan core */
darshan_core_register_record(
(void*)newname,
strlen(newname),
- 1,
DARSHAN_HDF5_MOD,
+ 1,
+ limit_flag,
&file_id,
NULL);
@@ -292,19 +296,15 @@ static struct hdf5_file_runtime* hdf5_file_by_name(const char *name)
return(file);
}
- if(hdf5_runtime->file_array_ndx < hdf5_runtime->file_array_size);
- {
- /* no existing record, assign a new file record from the global array */
- file = &(hdf5_runtime->file_runtime_array[hdf5_runtime->file_array_ndx]);
- file->file_record = &(hdf5_runtime->file_record_array[hdf5_runtime->file_array_ndx]);
- file->file_record->f_id = file_id;
- file->file_record->rank = my_rank;
-
- /* add new record to file hash table */
- HASH_ADD(hlink, hdf5_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
+ /* no existing record, assign a new file record from the global array */
+ file = &(hdf5_runtime->file_runtime_array[hdf5_runtime->file_array_ndx]);
+ file->file_record = &(hdf5_runtime->file_record_array[hdf5_runtime->file_array_ndx]);
+ file->file_record->f_id = file_id;
+ file->file_record->rank = my_rank;
- hdf5_runtime->file_array_ndx++;
- }
+ /* add new record to file hash table */
+ HASH_ADD(hlink, hdf5_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
+ hdf5_runtime->file_array_ndx++;
if(newname != name)
free(newname);
=====================================
darshan-runtime/lib/darshan-mpiio.c
=====================================
--- a/darshan-runtime/lib/darshan-mpiio.c
+++ b/darshan-runtime/lib/darshan-mpiio.c
@@ -878,6 +878,7 @@ static struct mpiio_file_runtime* mpiio_file_by_name(const char *name)
struct mpiio_file_runtime *file = NULL;
char *newname = NULL;
darshan_record_id file_id;
+ int limit_flag;
if(!mpiio_runtime || instrumentation_disabled)
return(NULL);
@@ -886,17 +887,20 @@ static struct mpiio_file_runtime* mpiio_file_by_name(const char *name)
if(!newname)
newname = (char*)name;
+ limit_flag = (mpiio_runtime->file_array_ndx >= mpiio_runtime->file_array_size);
+
/* get a unique id for this file from darshan core */
darshan_core_register_record(
(void*)newname,
strlen(newname),
- 1,
DARSHAN_MPIIO_MOD,
+ 1,
+ limit_flag,
&file_id,
NULL);
- /* if record is set to 0, darshan-core is out of space and will not
- * track this record, so we should avoid tracking it, too
+ /* the file record id is set to 0 if no memory is available for tracking
+ * new records -- just fall through and ignore this record
*/
if(file_id == 0)
{
@@ -914,19 +918,15 @@ static struct mpiio_file_runtime* mpiio_file_by_name(const char *name)
return(file);
}
- if(mpiio_runtime->file_array_ndx < mpiio_runtime->file_array_size);
- {
- /* no existing record, assign a new file record from the global array */
- file = &(mpiio_runtime->file_runtime_array[mpiio_runtime->file_array_ndx]);
- file->file_record = &(mpiio_runtime->file_record_array[mpiio_runtime->file_array_ndx]);
- file->file_record->f_id = file_id;
- file->file_record->rank = my_rank;
-
- /* add new record to file hash table */
- HASH_ADD(hlink, mpiio_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
+ /* no existing record, assign a new file record from the global array */
+ file = &(mpiio_runtime->file_runtime_array[mpiio_runtime->file_array_ndx]);
+ file->file_record = &(mpiio_runtime->file_record_array[mpiio_runtime->file_array_ndx]);
+ file->file_record->f_id = file_id;
+ file->file_record->rank = my_rank;
- mpiio_runtime->file_array_ndx++;
- }
+ /* add new record to file hash table */
+ HASH_ADD(hlink, mpiio_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
+ mpiio_runtime->file_array_ndx++;
if(newname != name)
free(newname);
=====================================
darshan-runtime/lib/darshan-null.c
=====================================
--- a/darshan-runtime/lib/darshan-null.c
+++ b/darshan-runtime/lib/darshan-null.c
@@ -264,6 +264,7 @@ static struct null_record_runtime* null_record_by_name(const char *name)
{
struct null_record_runtime *rec = NULL;
darshan_record_id rec_id;
+ int limit_flag;
/* Don't search for a record if the "NULL" module is not initialized or
* if instrumentation has been toggled off.
@@ -271,15 +272,27 @@ static struct null_record_runtime* null_record_by_name(const char *name)
if(!null_runtime || instrumentation_disabled)
return(NULL);
+ /* stop tracking new records if we are tracking our maximum count */
+ limit_flag = (null_runtime->rec_array_ndx >= null_runtime->rec_array_size);
+
/* get a unique record identifier for this record from darshan-core */
darshan_core_register_record(
(void*)name,
strlen(name),
- 1,
DARSHAN_NULL_MOD,
+ 1,
+ limit_flag,
&rec_id,
NULL);
+ /* the file record id is set to 0 if no memory is available for tracking
+ * new records -- just fall through and ignore this record
+ */
+ if(rec_id == 0)
+ {
+ return(NULL);
+ }
+
/* search the hash table for this file record, and return if found */
HASH_FIND(hlink, null_runtime->record_hash, &rec_id, sizeof(darshan_record_id), rec);
if(rec)
@@ -287,21 +300,17 @@ static struct null_record_runtime* null_record_by_name(const char *name)
return(rec);
}
- if(null_runtime->rec_array_ndx < null_runtime->rec_array_size);
- {
- /* no existing record, assign a new one from the global array */
- rec = &(null_runtime->runtime_record_array[null_runtime->rec_array_ndx]);
- rec->record_p = &(null_runtime->record_array[null_runtime->rec_array_ndx]);
+ /* no existing record, assign a new one from the global array */
+ rec = &(null_runtime->runtime_record_array[null_runtime->rec_array_ndx]);
+ rec->record_p = &(null_runtime->record_array[null_runtime->rec_array_ndx]);
- /* set the darshan record id and corresponding process rank for this record */
- rec->record_p->f_id = rec_id;
- rec->record_p->rank = my_rank;
+ /* set the darshan record id and corresponding process rank for this record */
+ rec->record_p->f_id = rec_id;
+ rec->record_p->rank = my_rank;
- /* add new record to file hash table */
- HASH_ADD(hlink, null_runtime->record_hash, record_p->f_id, sizeof(darshan_record_id), rec);
-
- null_runtime->rec_array_ndx++;
- }
+ /* add new record to file hash table */
+ HASH_ADD(hlink, null_runtime->record_hash, record_p->f_id, sizeof(darshan_record_id), rec);
+ null_runtime->rec_array_ndx++;
return(rec);
}
=====================================
darshan-runtime/lib/darshan-pnetcdf.c
=====================================
--- a/darshan-runtime/lib/darshan-pnetcdf.c
+++ b/darshan-runtime/lib/darshan-pnetcdf.c
@@ -268,6 +268,7 @@ static struct pnetcdf_file_runtime* pnetcdf_file_by_name(const char *name)
struct pnetcdf_file_runtime *file = NULL;
char *newname = NULL;
darshan_record_id file_id;
+ int limit_flag;
if(!pnetcdf_runtime || instrumentation_disabled)
return(NULL);
@@ -276,17 +277,20 @@ static struct pnetcdf_file_runtime* pnetcdf_file_by_name(const char *name)
if(!newname)
newname = (char*)name;
+ limit_flag = (pnetcdf_runtime->file_array_ndx >= pnetcdf_runtime->file_array_size);
+
/* get a unique id for this file from darshan core */
darshan_core_register_record(
(void*)newname,
strlen(newname),
- 1,
DARSHAN_PNETCDF_MOD,
+ 1,
+ limit_flag,
&file_id,
NULL);
- /* if record is set to 0, darshan-core is out of space and will not
- * track this record, so we should avoid tracking it, too
+ /* the file record id is set to 0 if no memory is available for tracking
+ * new records -- just fall through and ignore this record
*/
if(file_id == 0)
{
@@ -304,19 +308,15 @@ static struct pnetcdf_file_runtime* pnetcdf_file_by_name(const char *name)
return(file);
}
- if(pnetcdf_runtime->file_array_ndx < pnetcdf_runtime->file_array_size);
- {
- /* no existing record, assign a new file record from the global array */
- file = &(pnetcdf_runtime->file_runtime_array[pnetcdf_runtime->file_array_ndx]);
- file->file_record = &(pnetcdf_runtime->file_record_array[pnetcdf_runtime->file_array_ndx]);
- file->file_record->f_id = file_id;
- file->file_record->rank = my_rank;
-
- /* add new record to file hash table */
- HASH_ADD(hlink, pnetcdf_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
+ /* no existing record, assign a new file record from the global array */
+ file = &(pnetcdf_runtime->file_runtime_array[pnetcdf_runtime->file_array_ndx]);
+ file->file_record = &(pnetcdf_runtime->file_record_array[pnetcdf_runtime->file_array_ndx]);
+ file->file_record->f_id = file_id;
+ file->file_record->rank = my_rank;
- pnetcdf_runtime->file_array_ndx++;
- }
+ /* add new record to file hash table */
+ HASH_ADD(hlink, pnetcdf_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
+ pnetcdf_runtime->file_array_ndx++;
if(newname != name)
free(newname);
=====================================
darshan-runtime/lib/darshan-posix.c
=====================================
--- a/darshan-runtime/lib/darshan-posix.c
+++ b/darshan-runtime/lib/darshan-posix.c
@@ -1509,6 +1509,7 @@ static struct posix_file_runtime* posix_file_by_name(const char *name)
char *newname = NULL;
darshan_record_id file_id;
int file_alignment;
+ int limit_flag;
if(!posix_runtime || instrumentation_disabled)
return(NULL);
@@ -1517,17 +1518,20 @@ static struct posix_file_runtime* posix_file_by_name(const char *name)
if(!newname)
newname = (char*)name;
+ limit_flag = (posix_runtime->file_array_ndx >= posix_runtime->file_array_size);
+
/* get a unique id for this file from darshan core */
darshan_core_register_record(
(void*)newname,
strlen(newname),
- 1,
DARSHAN_POSIX_MOD,
+ 1,
+ limit_flag,
&file_id,
&file_alignment);
- /* if record is set to 0, darshan-core is out of space and will not
- * track this record, so we should avoid tracking it, too
+ /* the file record id is set to 0 if no memory is available for tracking
+ * new records -- just fall through and ignore this record
*/
if(file_id == 0)
{
@@ -1545,21 +1549,17 @@ static struct posix_file_runtime* posix_file_by_name(const char *name)
return(file);
}
- if(posix_runtime->file_array_ndx < posix_runtime->file_array_size);
- {
- /* no existing record, assign a new file record from the global array */
- file = &(posix_runtime->file_runtime_array[posix_runtime->file_array_ndx]);
- file->file_record = &(posix_runtime->file_record_array[posix_runtime->file_array_ndx]);
- file->file_record->f_id = file_id;
- file->file_record->rank = my_rank;
- file->file_record->counters[POSIX_MEM_ALIGNMENT] = darshan_mem_alignment;
- file->file_record->counters[POSIX_FILE_ALIGNMENT] = file_alignment;
-
- /* add new record to file hash table */
- HASH_ADD(hlink, posix_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
-
- posix_runtime->file_array_ndx++;
- }
+ /* no existing record, assign a new file record from the global array */
+ file = &(posix_runtime->file_runtime_array[posix_runtime->file_array_ndx]);
+ file->file_record = &(posix_runtime->file_record_array[posix_runtime->file_array_ndx]);
+ file->file_record->f_id = file_id;
+ file->file_record->rank = my_rank;
+ file->file_record->counters[POSIX_MEM_ALIGNMENT] = darshan_mem_alignment;
+ file->file_record->counters[POSIX_FILE_ALIGNMENT] = file_alignment;
+
+ /* add new record to file hash table */
+ HASH_ADD(hlink, posix_runtime->file_hash, file_record->f_id, sizeof(darshan_record_id), file);
+ posix_runtime->file_array_ndx++;
if(newname != name)
free(newname);
=====================================
darshan-util/darshan-job-summary/bin/darshan-job-summary.pl.in
=====================================
--- a/darshan-util/darshan-job-summary/bin/darshan-job-summary.pl.in
+++ b/darshan-util/darshan-job-summary/bin/darshan-job-summary.pl.in
@@ -97,7 +97,6 @@ while($line = <PARSE_OUT>)
$f_save = "";
($junk, $cmdline) = split(':', $line, 2);
- print("PRE: $cmdline\n");
# add escape characters if needed for special characters in
# command line
if ($cmdline =~ /<unknown args>/)
@@ -108,7 +107,6 @@ while($line = <PARSE_OUT>)
$cmdline = substr($cmdline, 0, -14);
}
$cmdline = encode('latex', $cmdline) . $f_save;
- print("POST: $cmdline\n");
}
elsif ($line =~ /^# nprocs: /)
{
@@ -135,7 +133,7 @@ while($line = <PARSE_OUT>)
($junk, $version) = split(':', $line, 2);
$version =~ s/^\s+//;
}
- elsif ($line =~ /^# \*WARNING\*: This Darshan log contains incomplete data!/)
+ elsif ($line =~ /^# \*WARNING\*: .* contains incomplete data!/)
{
$partial_flag = 1;
}
=====================================
darshan-util/darshan-parser.c
=====================================
--- a/darshan-util/darshan-parser.c
+++ b/darshan-util/darshan-parser.c
@@ -322,11 +322,6 @@ int main(int argc, char **argv)
}
/* warn user if this log file is incomplete */
- if(fd->partial_flag)
- printf("\n# *WARNING*: This Darshan log contains incomplete data!\n"
- "# This happens when an application creates\n"
- "# more records than Darshan can track.\n");
-
pdata.rank_cumul_io_time = malloc(sizeof(double)*job.nprocs);
pdata.rank_cumul_md_time = malloc(sizeof(double)*job.nprocs);
if (!pdata.rank_cumul_io_time || !pdata.rank_cumul_md_time)
@@ -370,6 +365,13 @@ int main(int argc, char **argv)
printf("# %s module data\n", darshan_module_names[i]);
printf("# *******************************************************\n");
+ /* print warning if this module only stored partial data */
+ if(DARSHAN_MOD_FLAG_ISSET(fd->partial_flag, i))
+ printf("\n# *WARNING*: The %s module contains incomplete data!\n"
+ "# This happens when a module runs out of\n"
+ "# memory to store new record data.\n",
+ darshan_module_names[i]);
+
if(mask & OPTION_BASE)
{
/* TODO: does each module print header of what each counter means??? */
View it on GitLab: https://xgitlab.cels.anl.gov/darshan/darshan/compare/48352d366d6530b22d3fc4a2821c8b7a7e3ea214...40d1dc03b7e336f5127f70a68c3a8069fd92536c
-------------- next part --------------
An HTML attachment was scrubbed...
URL: <http://lists.mcs.anl.gov/pipermail/darshan-commits/attachments/20151102/e93b67c9/attachment-0001.html>
More information about the Darshan-commits
mailing list