count = recv_count;
- tmp_buff = (char *) xbt_malloc(num_procs * recv_count * recv_extent);
+ tmp_buff = (char *) smpi_get_tmp_sendbuffer(num_procs * recv_count * recv_extent);
// perform a local copy
smpi_datatype_copy(send_ptr, send_count, send_type,
smpi_mpi_sendrecv(tmp_buff + (num_procs - rank) * recv_count * recv_extent,
rank * recv_count, recv_type, rank, tag, recv_ptr,
rank * recv_count, recv_type, rank, tag, comm, &status);
- free(tmp_buff);
+ smpi_free_tmp_buffer(tmp_buff);
return MPI_SUCCESS;
}
smpi_datatype_extent(recvtype, &recvtype_true_lb, &recvtype_true_extent);
- tmp_buf= (void*)xbt_malloc(total_count*(max(recvtype_true_extent,recvtype_extent)));
+ tmp_buf= (void*)smpi_get_tmp_sendbuffer(total_count*(max(recvtype_true_extent,recvtype_extent)));
/* adjust for potential negative lower bound in datatype */
tmp_buf = (void *)((char*)tmp_buf - recvtype_true_lb);
position += recvcounts[j];
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
smpi_datatype_extent(datatype, &true_lb, &true_extent);
extent = smpi_datatype_get_extent(datatype);
- tmp_buf_free= xbt_malloc(count * (MAX(extent, true_extent)));
+ tmp_buf_free= smpi_get_tmp_recvbuffer(count * (MAX(extent, true_extent)));
/* adjust for potential negative lower bound in datatype */
tmp_buf = (void *) ((char *) tmp_buf_free - true_lb);
MPI_STATUS_IGNORE);
}
}
- xbt_free(tmp_buf_free);
+ smpi_free_tmp_buffer(tmp_buf_free);
return (mpi_errno);
}
}
if (local_size != total_size) {
- void* sendtmpbuf = (char *)xbt_malloc(count*smpi_datatype_get_extent(datatype));
- smpi_datatype_copy(recvbuf, count, datatype,sendtmpbuf, count, datatype);
+ void* sendtmpbuf = (char *)smpi_get_tmp_sendbuffer(count*smpi_datatype_get_extent(datatype));
+ smpi_datatype_copy(recvbuf, count, datatype,sendtmpbuf, count, datatype);
/* inter-node allreduce */
if(MV2_Allreduce_function == &MPIR_Allreduce_pt2pt_rd_MV2){
mpi_errno =
MPIR_Allreduce_pt2pt_rs_MV2(sendtmpbuf, recvbuf, count, datatype, op,
leader_comm);
}
- xbt_free(sendtmpbuf);
+ smpi_free_tmp_buffer(sendtmpbuf);
}
} else {
/* insert the first reduce here */
rank = smpi_comm_rank(comm);
extent = smpi_datatype_get_extent(dtype);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
smpi_datatype_copy(sbuff, count, dtype, rbuff, count, dtype);
smpi_mpi_recv(rbuff, count, dtype, rank + 1, tag, comm, &status);
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
send_size = (count + nprocs) / nprocs;
newcnt = send_size * nprocs;
- recv = (void *) xbt_malloc(extent * newcnt);
- tmp_buf = (void *) xbt_malloc(extent * newcnt);
+ recv = (void *) smpi_get_tmp_recvbuffer(extent * newcnt);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(extent * newcnt);
memcpy(recv, sbuff, extent * count);
mpi_coll_allgather_fun(tmp_buf, recv_cnt, dtype, recv, recv_cnt, dtype, comm);
memcpy(rbuff, recv, count * extent);
- free(recv);
- free(tmp_buf);
+ smpi_free_tmp_buffer(recv);
+ smpi_free_tmp_buffer(tmp_buf);
}
else {
- tmp_buf = (void *) xbt_malloc(extent * count);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(extent * count);
memcpy(rbuff, sbuff, count * extent);
mask = pof2 / 2;
share = count / pof2;
memcpy(tmp_buf, (char *) rbuff + recv_idx * extent, recv_cnt * extent);
mpi_coll_allgather_fun(tmp_buf, recv_cnt, dtype, rbuff, recv_cnt, dtype, comm);
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
}
return MPI_SUCCESS;
send_size = (count + nprocs) / nprocs;
nbytes = send_size * s_extent;
- send = (void *) xbt_malloc(s_extent * send_size * nprocs);
- recv = (void *) xbt_malloc(s_extent * send_size * nprocs);
- tmp = (void *) xbt_malloc(nbytes);
+ send = (void *) smpi_get_tmp_sendbuffer(s_extent * send_size * nprocs);
+ recv = (void *) smpi_get_tmp_recvbuffer(s_extent * send_size * nprocs);
+ tmp = (void *) smpi_get_tmp_sendbuffer(nbytes);
memcpy(send, sbuff, s_extent * count);
mpi_coll_allgather_fun(tmp, send_size, dtype, recv, send_size, dtype, comm);
memcpy(rbuff, recv, count * s_extent);
- free(recv);
- free(tmp);
- free(send);
+ smpi_free_tmp_buffer(recv);
+ smpi_free_tmp_buffer(tmp);
+ smpi_free_tmp_buffer(send);
} else {
send = sbuff;
send_size = count / nprocs;
nbytes = send_size * s_extent;
r_offset = rank * nbytes;
- recv = (void *) xbt_malloc(s_extent * send_size * nprocs);
+ recv = (void *) smpi_get_tmp_recvbuffer(s_extent * send_size * nprocs);
mpi_coll_alltoall_fun(send, send_size, dtype, recv, send_size, dtype, comm);
mpi_coll_allgather_fun((char *) rbuff + r_offset, send_size, dtype, rbuff, send_size,
dtype, comm);
- free(recv);
+ smpi_free_tmp_buffer(recv);
}
return MPI_SUCCESS;
rank=smpi_comm_rank(comm);
smpi_datatype_extent(dtype, &lb, &extent);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
smpi_mpi_sendrecv(sbuff, count, dtype, rank, 500,
rbuff, count, dtype, rank, 500, comm, &status);
smpi_mpi_recv(rbuff, count, dtype, rank + 1, tag, comm, &status);
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
rank = smpi_comm_rank(comm);
MPI_Aint extent;
extent = smpi_datatype_get_extent(dtype);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
int intra_rank, inter_rank;
intra_rank = rank % num_core;
}
} // for phase
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
rank=smpi_comm_rank(comm);
MPI_Aint extent, lb;
smpi_datatype_extent(dtype, &lb, &extent);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
/* compute intra and inter ranking */
int intra_rank, inter_rank;
mask >>= 1;
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
rank = smpi_comm_rank(comm);
MPI_Aint extent;
extent = smpi_datatype_get_extent(dtype);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
/* compute intra and inter ranking */
int intra_rank, inter_rank;
mask >>= 1;
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
rank = smpi_comm_rank(comm);
MPI_Aint extent;
extent = smpi_datatype_get_extent(dtype);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
int intra_rank, inter_rank;
intra_rank = rank % num_core;
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
rank = smpi_comm_rank(comm);
MPI_Aint extent;
extent = smpi_datatype_get_extent(dtype);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
int intra_rank, inter_rank;
intra_rank = rank % num_core;
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
rank = smpi_comm_rank(comm);
MPI_Aint extent;
extent = smpi_datatype_get_extent(dtype);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
int intra_rank, inter_rank;
intra_rank = rank % num_core;
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
block_size = extent * send_count;
- tmp_buff1 = (char *) xbt_malloc(block_size * num_procs * Y);
- tmp_buff2 = (char *) xbt_malloc(block_size * Y);
+ tmp_buff1 = (char *) smpi_get_tmp_sendbuffer(block_size * num_procs * Y);
+ tmp_buff2 = (char *) smpi_get_tmp_recvbuffer(block_size * Y);
num_reqs = X;
if (Y > X)
smpi_mpi_waitall(X - 1, reqs, statuses);
free(reqs);
free(statuses);
- free(tmp_buff1);
- free(tmp_buff2);
+ smpi_free_tmp_buffer(tmp_buff1);
+ smpi_free_tmp_buffer(tmp_buff2);
return MPI_SUCCESS;
}
block_size = extent * send_count;
- tmp_buff1 = (char *) xbt_malloc(block_size * num_procs * two_dsize);
- tmp_buff2 = (char *) xbt_malloc(block_size * two_dsize);
+ tmp_buff1 = (char *) smpi_get_tmp_sendbuffer(block_size * num_procs * two_dsize);
+ tmp_buff2 = (char *) smpi_get_tmp_recvbuffer(block_size * two_dsize);
statuses = (MPI_Status *) xbt_malloc(num_reqs * sizeof(MPI_Status));
reqs = (MPI_Request *) xbt_malloc(num_reqs * sizeof(MPI_Request));
free(reqs);
free(statuses);
- free(tmp_buff1);
- free(tmp_buff2);
+ smpi_free_tmp_buffer(tmp_buff1);
+ smpi_free_tmp_buffer(tmp_buff2);
return MPI_SUCCESS;
}
extent = smpi_datatype_get_extent(recv_type);
- tmp_buff = (char *) xbt_malloc(num_procs * recv_count * extent);
+ tmp_buff = (char *) smpi_get_tmp_sendbuffer(num_procs * recv_count * extent);
disps = (int *) xbt_malloc(sizeof(int) * num_procs);
blocks_length = (int *) xbt_malloc(sizeof(int) * num_procs);
recv_ptr + (num_procs - i - 1) * recv_count * extent,
recv_count, recv_type, rank, tag, comm, &status);
- free(tmp_buff);
+ smpi_free_tmp_buffer(tmp_buff);
return MPI_SUCCESS;
}
max_size = num_procs * recv_increment;
- tmp_buff = (char *) xbt_malloc(max_size);
+ tmp_buff = (char *) smpi_get_tmp_sendbuffer(max_size);
curr_size = send_count * num_procs;
send_count, send_type, rank, tag,
recv_ptr + (i * recv_count * extent),
recv_count, recv_type, rank, tag, comm, &status);
- free(tmp_buff);
+ smpi_free_tmp_buffer(tmp_buff);
return MPI_SUCCESS;
}
) {
if (!is_contig || !is_homogeneous) {
- tmp_buf=(void *)xbt_malloc(nbytes);
+ tmp_buf=(void *)smpi_get_tmp_sendbuffer(nbytes);
/* TODO: Pipeline the packing and communication */
// position = 0;
if (local_rank == 0) {
/* Node leader, allocate tmp_buffer */
if (rank == root) {
- tmp_buf = xbt_malloc(recvcnt * MAX(recvtype_extent,
+ tmp_buf = smpi_get_tmp_recvbuffer(recvcnt * MAX(recvtype_extent,
recvtype_true_extent) * local_size);
} else {
- tmp_buf = xbt_malloc(sendcnt * MAX(sendtype_extent,
+ tmp_buf = smpi_get_tmp_sendbuffer(sendcnt * MAX(sendtype_extent,
sendtype_true_extent) *
local_size);
}
* leader and this process's rank in the leader_comm
* is the same as leader_root */
if(rank == root) {
- leader_gather_buf = xbt_malloc(recvcnt *
+ leader_gather_buf = smpi_get_tmp_recvbuffer(recvcnt *
MAX(recvtype_extent,
recvtype_true_extent) *
comm_size);
} else {
- leader_gather_buf = xbt_malloc(sendcnt *
+ leader_gather_buf = smpi_get_tmp_sendbuffer(sendcnt *
MAX(sendtype_extent,
sendtype_true_extent) *
comm_size);
if (leader_comm_rank == leader_root && root != leader_of_root) {
/* The root of the Gather operation is not a node-level leader
*/
- leader_gather_buf = xbt_malloc(nbytes * comm_size);
+ leader_gather_buf = smpi_get_tmp_sendbuffer(nbytes * comm_size);
if (leader_gather_buf == NULL) {
mpi_errno = MPI_ERR_OTHER;
return mpi_errno;
/* check if multiple threads are calling this collective function */
if (local_rank == 0 ) {
if (tmp_buf != NULL) {
- xbt_free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
}
if (leader_gather_buf != NULL) {
- xbt_free(leader_gather_buf);
+ smpi_free_tmp_buffer(leader_gather_buf);
}
}
*/
char *tmp_buf;
- tmp_buf = (char *) xbt_malloc(count * extent);
+ tmp_buf = (char *) smpi_get_tmp_sendbuffer(count * extent);
smpi_mpi_sendrecv(buf, count, datatype, rank, tag, rbuf, count, datatype, rank,
tag, comm, &status);
smpi_op_apply(op, tmp_buf, rbuf, &count, &datatype);
smpi_mpi_send(rbuf, count, datatype, to, tag, comm);
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
}
char *tmp_buf;
- tmp_buf = (char *) xbt_malloc(count * extent);
+ tmp_buf = (char *) smpi_get_tmp_sendbuffer(count * extent);
smpi_mpi_sendrecv(buf, count, datatype, rank, tag, rbuf, count, datatype, rank,
tag, comm, &status);
comm);
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return MPI_SUCCESS;
}
extent = smpi_datatype_get_extent(datatype);
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
int is_commutative = smpi_op_is_commute(op);
mask = 1;
/* If I'm not the root, then my recvbuf may not be valid, therefore
I have to allocate a temporary one */
if (rank != root) {
- recvbuf = (void *) malloc(count*(max(extent,true_extent)));
+ recvbuf = (void *) smpi_get_tmp_recvbuffer(count*(max(extent,true_extent)));
recvbuf = (void *)((char*)recvbuf - true_lb);
}
if ((rank != root) || (sendbuf != MPI_IN_PLACE)) {
}
if (rank != root) {
- xbt_free(recvbuf);
+ smpi_free_tmp_buffer(recvbuf);
}
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
return 0;
}
/* Finally, fill up the src array */
if(recv_iter > 0) {
- knomial_reduce_src_array = xbt_malloc(sizeof(int)*recv_iter);
+ knomial_reduce_src_array = smpi_get_tmp_sendbuffer(sizeof(int)*recv_iter);
}
mask = orig_mask;
is_commutative = smpi_op_is_commute(op);
if (rank != root) {
- recvbuf=(void *)xbt_malloc(count*(MAX(extent,true_extent)));
+ recvbuf=(void *)smpi_get_tmp_recvbuffer(count*(MAX(extent,true_extent)));
recvbuf = (void *)((char*)recvbuf - true_lb);
}
&dst, &expected_send_count, &expected_recv_count, &src_array);
if(expected_recv_count > 0 ) {
- tmp_buf = xbt_malloc(sizeof(void *)*expected_recv_count);
+ tmp_buf = smpi_get_tmp_recvbuffer(sizeof(void *)*expected_recv_count);
requests = xbt_malloc(sizeof(MPI_Request)*expected_recv_count);
for(k=0; k < expected_recv_count; k++ ) {
- tmp_buf[k] = xbt_malloc(count*(MAX(extent,true_extent)));
+ tmp_buf[k] = smpi_get_tmp_sendbuffer(count*(MAX(extent,true_extent)));
tmp_buf[k] = (void *)((char*)tmp_buf[k] - true_lb);
}
}
for(k=0; k < expected_recv_count; k++ ) {
- xbt_free(tmp_buf[k]);
+ smpi_free_tmp_buffer(tmp_buf[k]);
}
- xbt_free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
xbt_free(requests);
}
if(src_array != NULL) {
- xbt_free(src_array);
+ smpi_free_tmp_buffer(src_array);
}
if(rank != root) {
if (stride <= MV2_INTRA_SHMEM_REDUCE_MSG &&
is_commutative == 1) {
if (local_rank == 0 ) {
- tmp_buf=(void *)xbt_malloc( count *
+ tmp_buf=(void *)smpi_get_tmp_sendbuffer( count *
(MAX(extent, true_extent)));
tmp_buf = (void *) ((char *) tmp_buf - true_lb);
}
}
leader_comm_size = smpi_comm_size(leader_comm);
leader_comm_rank = smpi_comm_rank(leader_comm);
- tmp_buf=(void *)xbt_malloc(count *
+ tmp_buf=(void *)smpi_get_tmp_sendbuffer(count *
(MAX(extent, true_extent)));
tmp_buf = (void *) ((char *) tmp_buf - true_lb);
}
out_buf = recvbuf;
} else {
- in_buf = (char *)xbt_malloc(count*
+ in_buf = (char *)smpi_get_tmp_sendbuffer(count*
smpi_datatype_get_extent(datatype));
smpi_datatype_copy(tmp_buf, count, datatype,
in_buf, count, datatype);
out_buf = recvbuf;
}
} else {
- in_buf = (char *)xbt_malloc(count*
+ in_buf = (char *)smpi_get_tmp_sendbuffer(count*
smpi_datatype_get_extent(datatype));
smpi_datatype_copy(tmp_buf, count, datatype,
in_buf, count, datatype);
accumbuf = (char*)recvbuf;
if( (NULL == accumbuf) || (root != rank) ) {
/* Allocate temporary accumulator buffer. */
- accumbuf_free = (char*)malloc(true_extent +
+ accumbuf_free = (char*)smpi_get_tmp_sendbuffer(true_extent +
(original_count - 1) * extent);
if (accumbuf_free == NULL) {
line = __LINE__; ret = -1; goto error_hndl;
}
/* Allocate two buffers for incoming segments */
real_segment_size = true_extent + (count_by_segment - 1) * extent;
- inbuf_free[0] = (char*) malloc(real_segment_size);
+ inbuf_free[0] = (char*) smpi_get_tmp_recvbuffer(real_segment_size);
if( inbuf_free[0] == NULL ) {
line = __LINE__; ret = -1; goto error_hndl;
}
/* if there is chance to overlap communication -
allocate second buffer */
if( (num_segments > 1) || (tree->tree_nextsize > 1) ) {
- inbuf_free[1] = (char*) malloc(real_segment_size);
+ inbuf_free[1] = (char*) smpi_get_tmp_recvbuffer(real_segment_size);
if( inbuf_free[1] == NULL ) {
line = __LINE__; ret = -1; goto error_hndl;
}
} /* end of for each segment */
/* clean up */
- if( inbuf_free[0] != NULL) free(inbuf_free[0]);
- if( inbuf_free[1] != NULL) free(inbuf_free[1]);
- if( accumbuf_free != NULL ) free(accumbuf_free);
+ smpi_free_tmp_buffer(inbuf_free[0]);
+ smpi_free_tmp_buffer(inbuf_free[1]);
+ smpi_free_tmp_buffer(accumbuf_free);
}
/* leaf nodes
text=smpi_datatype_get_extent(datatype);
if ((root == rank) && (MPI_IN_PLACE == sendbuf)) {
- tmpbuf = (char *) malloc(text + (count - 1) * ext);
+ tmpbuf = (char *) smpi_get_tmp_sendbuffer(text + (count - 1) * ext);
if (NULL == tmpbuf) {
return MPI_ERR_INTERN;
}
(char*)tmpbuf, count, datatype);
use_this_sendbuf = tmpbuf;
} else if (io_root == rank) {
- tmpbuf = (char *) malloc(text + (count - 1) * ext);
+ tmpbuf = (char *) smpi_get_tmp_recvbuffer(text + (count - 1) * ext);
if (NULL == tmpbuf) {
return MPI_ERR_INTERN;
}
COLL_TAG_REDUCE, comm,
MPI_STATUS_IGNORE);
if (MPI_IN_PLACE == sendbuf) {
- free(use_this_sendbuf);
+ smpi_free_tmp_buffer(use_this_sendbuf);
}
} else if (io_root == rank) {
smpi_mpi_send(use_this_recvbuf, count, datatype, root,
COLL_TAG_REDUCE,
comm);
- free(use_this_recvbuf);
+ smpi_free_tmp_buffer(use_this_recvbuf);
}
}
if (MPI_IN_PLACE == sbuf) {
sbuf = rbuf;
- inplace_temp = (char*)malloc(true_extent + (count - 1) * extent);
+ inplace_temp = (char*)smpi_get_tmp_recvbuffer(true_extent + (count - 1) * extent);
if (NULL == inplace_temp) {
return -1;
}
}
if (size > 1) {
- free_buffer = (char*)malloc(true_extent + (count - 1) * extent);
+ free_buffer = (char*)smpi_get_tmp_recvbuffer(true_extent + (count - 1) * extent);
pml_buffer = free_buffer - lb;
}
if (NULL != inplace_temp) {
smpi_datatype_copy(inplace_temp, count, dtype,(char*)sbuf
,count , dtype);
- free(inplace_temp);
+ smpi_free_tmp_buffer(inplace_temp);
}
if (NULL != free_buffer) {
- free(free_buffer);
+ smpi_free_tmp_buffer(free_buffer);
}
/* All done */
/* If I'm not the root, then my recvbuf may not be valid, therefore
I have to allocate a temporary one */
if (rank != root && !recvbuf) {
- recvbuf = (void *)xbt_malloc(count * extent);
+ recvbuf = (void *)smpi_get_tmp_recvbuffer(count * extent);
}
/* find nearest power-of-two less than or equal to comm_size */
pof2 = 1;
if (count < comm_size) {
new_count = comm_size;
- send_ptr = (void *) xbt_malloc(new_count * extent);
- recv_ptr = (void *) xbt_malloc(new_count * extent);
- tmp_buf = (void *) xbt_malloc(new_count * extent);
+ send_ptr = (void *) smpi_get_tmp_sendbuffer(new_count * extent);
+ recv_ptr = (void *) smpi_get_tmp_recvbuffer(new_count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(new_count * extent);
memcpy(send_ptr, sendbuf, extent * count);
//if ((rank != root))
}
}
memcpy(recvbuf, recv_ptr, extent * count);
- free(send_ptr);
- free(recv_ptr);
+ smpi_free_tmp_buffer(send_ptr);
+ smpi_free_tmp_buffer(recv_ptr);
}
else /* (count >= comm_size) */ {
- tmp_buf = (void *) xbt_malloc(count * extent);
+ tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
//if ((rank != root))
smpi_mpi_sendrecv(sendbuf, count, datatype, rank, tag,
}
}
if (tmp_buf)
- free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
if (cnts)
free(cnts);
if (disps)
}
/* allocate temporary buffer to store incoming data */
- tmp_recvbuf = (void*)xbt_malloc(recvcounts[rank]*(max(true_extent,extent))+1);
+ tmp_recvbuf = (void*)smpi_get_tmp_recvbuffer(recvcounts[rank]*(max(true_extent,extent))+1);
/* adjust for potential negative lower bound in datatype */
tmp_recvbuf = (void *)((char*)tmp_recvbuf - true_lb);
}
xbt_free(disps);
- xbt_free(tmp_recvbuf);
+ smpi_free_tmp_buffer(tmp_recvbuf);
return MPI_SUCCESS;
}
block_size = recvcounts[0];
total_count = block_size * comm_size;
- tmp_buf0=( void *)xbt_malloc( true_extent * total_count);
- tmp_buf1=( void *)xbt_malloc( true_extent * total_count);
+ tmp_buf0=( void *)smpi_get_tmp_sendbuffer( true_extent * total_count);
+ tmp_buf1=( void *)smpi_get_tmp_recvbuffer( true_extent * total_count);
void *tmp_buf0_save=tmp_buf0;
void *tmp_buf1_save=tmp_buf1;
result_ptr = (char *)(buf0_was_inout ? tmp_buf0 : tmp_buf1) + recv_offset * true_extent;
mpi_errno = smpi_datatype_copy(result_ptr, size, datatype,
recvbuf, size, datatype);
- xbt_free(tmp_buf0_save);
- xbt_free(tmp_buf1_save);
+ smpi_free_tmp_buffer(tmp_buf0_save);
+ smpi_free_tmp_buffer(tmp_buf1_save);
if (mpi_errno) return(mpi_errno);
return MPI_SUCCESS;
}
/* noncommutative and (non-pof2 or block irregular), use recursive doubling. */
/* need to allocate temporary buffer to receive incoming data*/
- tmp_recvbuf= (void *) xbt_malloc( total_count*(max(true_extent,extent)));
+ tmp_recvbuf= (void *) smpi_get_tmp_recvbuffer( total_count*(max(true_extent,extent)));
/* adjust for potential negative lower bound in datatype */
tmp_recvbuf = (void *)((char*)tmp_recvbuf - true_lb);
/* need to allocate another temporary buffer to accumulate
results */
- tmp_results = (void *)xbt_malloc( total_count*(max(true_extent,extent)));
+ tmp_results = (void *)smpi_get_tmp_sendbuffer( total_count*(max(true_extent,extent)));
/* adjust for potential negative lower bound in datatype */
tmp_results = (void *)((char*)tmp_results - true_lb);
if (mpi_errno) return(mpi_errno);
xbt_free(disps);
- xbt_free(tmp_recvbuf);
- xbt_free(tmp_results);
+ smpi_free_tmp_buffer(tmp_recvbuf);
+ smpi_free_tmp_buffer(tmp_results);
return MPI_SUCCESS;
}
}
/* Allocate temporary receive buffer. */
-#ifndef WIN32
- if(_xbt_replay_is_active()){
- recv_buf_free = (char*) SMPI_SHARED_MALLOC(buf_size);
- }else
-#endif
- recv_buf_free = (char*) xbt_malloc(buf_size);
+ recv_buf_free = (char*) smpi_get_tmp_recvbuffer(buf_size);
recv_buf = recv_buf_free - lb;
if (NULL == recv_buf_free) {
}
/* allocate temporary buffer for results */
-#ifndef WIN32
- if(_xbt_replay_is_active()){
- result_buf_free = (char*) SMPI_SHARED_MALLOC(buf_size);
- }else
-#endif
- result_buf_free = (char*) xbt_malloc(buf_size);
+ result_buf_free = (char*) smpi_get_tmp_sendbuffer(buf_size);
result_buf = result_buf_free - lb;
cleanup:
if (NULL != disps) xbt_free(disps);
- if (!_xbt_replay_is_active()){
- if (NULL != recv_buf_free) xbt_free(recv_buf_free);
- if (NULL != result_buf_free) xbt_free(result_buf_free);
- }
-#ifndef WIN32
- else{
- if (NULL != recv_buf_free) SMPI_SHARED_FREE(recv_buf_free);
- if (NULL != result_buf_free) SMPI_SHARED_FREE(result_buf_free);
- }
-#endif
+ if (NULL != recv_buf_free) smpi_free_tmp_buffer(recv_buf_free);
+ if (NULL != result_buf_free) smpi_free_tmp_buffer(result_buf_free);
+
return err;
}
max_real_segsize = true_extent + (ptrdiff_t)(max_block_count - 1) * extent;
- accumbuf_free = (char*)xbt_malloc(true_extent + (ptrdiff_t)(total_count - 1) * extent);
+ accumbuf_free = (char*)smpi_get_tmp_recvbuffer(true_extent + (ptrdiff_t)(total_count - 1) * extent);
if (NULL == accumbuf_free) { ret = -1; line = __LINE__; goto error_hndl; }
accumbuf = accumbuf_free - lb;
- inbuf_free[0] = (char*)xbt_malloc(max_real_segsize);
+ inbuf_free[0] = (char*)smpi_get_tmp_sendbuffer(max_real_segsize);
if (NULL == inbuf_free[0]) { ret = -1; line = __LINE__; goto error_hndl; }
inbuf[0] = inbuf_free[0] - lb;
if (size > 2) {
- inbuf_free[1] = (char*)xbt_malloc(max_real_segsize);
+ inbuf_free[1] = (char*)smpi_get_tmp_sendbuffer(max_real_segsize);
if (NULL == inbuf_free[1]) { ret = -1; line = __LINE__; goto error_hndl; }
inbuf[1] = inbuf_free[1] - lb;
}
if (ret < 0) { line = __LINE__; goto error_hndl; }
if (NULL != displs) xbt_free(displs);
- if (NULL != accumbuf_free) xbt_free(accumbuf_free);
- if (NULL != inbuf_free[0]) xbt_free(inbuf_free[0]);
- if (NULL != inbuf_free[1]) xbt_free(inbuf_free[1]);
+ if (NULL != accumbuf_free) smpi_free_tmp_buffer(accumbuf_free);
+ if (NULL != inbuf_free[0]) smpi_free_tmp_buffer(inbuf_free[0]);
+ if (NULL != inbuf_free[1]) smpi_free_tmp_buffer(inbuf_free[1]);
return MPI_SUCCESS;
XBT_DEBUG( "%s:%4d\tRank %d Error occurred %d\n",
__FILE__, line, rank, ret);
if (NULL != displs) xbt_free(displs);
- if (NULL != accumbuf_free) xbt_free(accumbuf_free);
- if (NULL != inbuf_free[0]) xbt_free(inbuf_free[0]);
- if (NULL != inbuf_free[1]) xbt_free(inbuf_free[1]);
+ if (NULL != accumbuf_free) smpi_free_tmp_buffer(accumbuf_free);
+ if (NULL != inbuf_free[0]) smpi_free_tmp_buffer(inbuf_free[0]);
+ if (NULL != inbuf_free[1]) smpi_free_tmp_buffer(inbuf_free[1]);
return ret;
}
if (local_rank == 0) {
/* Node leader, allocate tmp_buffer */
- tmp_buf = xbt_malloc(nbytes * local_size);
+ tmp_buf = smpi_get_tmp_sendbuffer(nbytes * local_size);
}
leader_comm = smpi_comm_get_leaders_comm(comm);
&& (leader_of_root == rank)) {
/* The root of the scatter operation is not the node leader. Recv
* data from the node leader */
- leader_scatter_buf = xbt_malloc(nbytes * comm_size);
+ leader_scatter_buf = smpi_get_tmp_sendbuffer(nbytes * comm_size);
smpi_mpi_recv(leader_scatter_buf, nbytes * comm_size, MPI_BYTE,
root, COLL_TAG_SCATTER, comm, &status);
/* check if multiple threads are calling this collective function */
if (comm_size != local_size && local_rank == 0) {
- xbt_free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
if (leader_of_root == rank && root != rank) {
- xbt_free(leader_scatter_buf);
+ smpi_free_tmp_buffer(leader_scatter_buf);
}
}
return (mpi_errno);
if (local_rank == 0) {
/* Node leader, allocate tmp_buffer */
- tmp_buf = xbt_malloc(nbytes * local_size);
+ tmp_buf = smpi_get_tmp_sendbuffer(nbytes * local_size);
}
leader_comm = smpi_comm_get_leaders_comm(comm);
int* leaders_map = smpi_comm_get_leaders_map(comm);
&& (leader_of_root == rank)) {
/* The root of the scatter operation is not the node leader. Recv
* data from the node leader */
- leader_scatter_buf = xbt_malloc(nbytes * comm_size);
+ leader_scatter_buf = smpi_get_tmp_sendbuffer(nbytes * comm_size);
smpi_mpi_recv(leader_scatter_buf, nbytes * comm_size, MPI_BYTE,
root, COLL_TAG_SCATTER, comm, &status);
}
/* check if multiple threads are calling this collective function */
if (comm_size != local_size && local_rank == 0) {
- xbt_free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
if (leader_of_root == rank && root != rank) {
- xbt_free(leader_scatter_buf);
+ smpi_free_tmp_buffer(leader_scatter_buf);
}
}
/* Copyright (c) 2009-2010, 2013-2014. The SimGrid Team.
* All rights reserved. */
-/* This program is xbt_free software; you can redistribute it and/or modify it
+/* This program is free software; you can redistribute it and/or modify it
* under the terms of the license (GNU LGPL) which comes with this package. */
#include "colls_private.h"
mv2_alltoall_thresholds_table[conf_index][range].in_place_algo_table[range_threshold].min
||nbytes > mv2_alltoall_thresholds_table[conf_index][range].in_place_algo_table[range_threshold].max
) {
- tmp_buf = (char *)xbt_malloc( comm_size * recvcount * recvtype_size );
+ tmp_buf = (char *)smpi_get_tmp_sendbuffer( comm_size * recvcount * recvtype_size );
mpi_errno = smpi_datatype_copy((char *)recvbuf,
comm_size*recvcount, recvtype,
(char *)tmp_buf,
mpi_errno = MV2_Alltoall_function(tmp_buf, recvcount, recvtype,
recvbuf, recvcount, recvtype,
comm );
- xbt_free(tmp_buf);
+ smpi_free_tmp_buffer(tmp_buf);
} else {
mpi_errno = MPIR_Alltoall_inplace_MV2(sendbuf, sendcount, sendtype,
recvbuf, recvcount, recvtype,
#endif
if (two_level_bcast == 1) {
if (!is_contig || !is_homogeneous) {
- tmp_buf=(void *)xbt_malloc(nbytes);
+ tmp_buf=(void *)smpi_get_tmp_sendbuffer(nbytes);
/* position = 0;*/
/* if (rank == root) {*/
void smpi_bench_begin(void);
void smpi_bench_end(void);
+void* smpi_get_tmp_sendbuffer(int size);
+void* smpi_get_tmp_recvbuffer(int size);
+void smpi_free_tmp_buffer(void* buf);
+
+
// f77 wrappers
void mpi_init_(int*);
displs[i] = count;
count += recvcounts[i];
}
- tmpbuf=(void*)xbt_malloc(count*smpi_datatype_get_extent(datatype));
+ tmpbuf=(void*)smpi_get_tmp_sendbuffer(count*smpi_datatype_get_extent(datatype));
+
mpi_coll_reduce_fun(sendbuf, tmpbuf, count, datatype, op, 0, comm);
smpi_mpi_scatterv(tmpbuf, recvcounts, displs, datatype, recvbuf,
recvcounts[rank], datatype, 0, comm);
xbt_free(displs);
- xbt_free(tmpbuf);
+ smpi_free_tmp_buffer(tmpbuf);
}
void smpi_mpi_gatherv(void *sendbuf, int sendcount, MPI_Datatype sendtype,
if(src != root) {
// FIXME: possibly overkill we we have contiguous/noncontiguous data
// mapping...
- tmpbufs[index] = xbt_malloc(count * dataext);
+ if (!_xbt_replay_is_active())
+ tmpbufs[index] = xbt_malloc(count * dataext);
+ else
+ tmpbufs[index] = smpi_get_tmp_sendbuffer(count * dataext);
requests[index] =
smpi_irecv_init(tmpbufs[index], count, datatype, src,
system_tag, comm);
if(op) /* op can be MPI_OP_NULL that does nothing */
smpi_op_apply(op, tmpbufs[index], recvbuf, &count, &datatype);
}
- for(index = 0; index < size - 1; index++) {
- xbt_free(tmpbufs[index]);
- }
+ for(index = 0; index < size - 1; index++) {
+ smpi_free_tmp_buffer(tmpbufs[index]);
+ }
xbt_free(tmpbufs);
xbt_free(requests);
}
//allocate a single buffer for all sends, growing it if needed
-static void* get_sendbuffer(int size){
+void* smpi_get_tmp_sendbuffer(int size){
+ if (!_xbt_replay_is_active())
+ return xbt_malloc(size);
if (sendbuffer_size<size){
sendbuffer=xbt_realloc(sendbuffer,size);
sendbuffer_size=size;
return sendbuffer;
}
//allocate a single buffer for all recv
-static void* get_recvbuffer(int size){
+void* smpi_get_tmp_recvbuffer(int size){
+ if (!_xbt_replay_is_active())
+ return xbt_malloc(size);
if (recvbuffer_size<size){
recvbuffer=xbt_realloc(recvbuffer,size);
recvbuffer_size=size;
return sendbuffer;
}
+void smpi_free_tmp_buffer(void* buf){
+ if (!_xbt_replay_is_active())
+ xbt_free(buf);
+}
+
/* Helper function */
static double parse_double(const char *string)
{
MPI_CURRENT_TYPE=MPI_DEFAULT_TYPE;
MPI_CURRENT_TYPE2=MPI_DEFAULT_TYPE;
}
- void *send = get_sendbuffer(send_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE));
- void *recv = get_recvbuffer(recv_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
+ void *send = smpi_get_tmp_sendbuffer(send_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE));
+ void *recv = smpi_get_tmp_recvbuffer(recv_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
#ifdef HAVE_TRACING
int rank = smpi_process_index();
MPI_CURRENT_TYPE=MPI_DEFAULT_TYPE;
MPI_CURRENT_TYPE2=MPI_DEFAULT_TYPE;
}
- void *send = get_sendbuffer(send_size* smpi_datatype_size(MPI_CURRENT_TYPE));
+ void *send = smpi_get_tmp_sendbuffer(send_size* smpi_datatype_size(MPI_CURRENT_TYPE));
void *recv = NULL;
int root=atoi(action[4]);
int rank = smpi_process_index();
if(rank==root)
- recv = get_recvbuffer(recv_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
+ recv = smpi_get_tmp_recvbuffer(recv_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
#ifdef HAVE_TRACING
instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
MPI_CURRENT_TYPE=MPI_DEFAULT_TYPE;
MPI_CURRENT_TYPE2=MPI_DEFAULT_TYPE;
}
- void *send = get_sendbuffer(send_size* smpi_datatype_size(MPI_CURRENT_TYPE));
+ void *send = smpi_get_tmp_sendbuffer(send_size* smpi_datatype_size(MPI_CURRENT_TYPE));
void *recv = NULL;
for(i=0;i<comm_size;i++) {
recvcounts[i] = atoi(action[i+3]);
int rank = smpi_process_index();
if(rank==root)
- recv = get_recvbuffer(recv_sum* smpi_datatype_size(MPI_CURRENT_TYPE2));
+ recv = smpi_get_tmp_recvbuffer(recv_sum* smpi_datatype_size(MPI_CURRENT_TYPE2));
#ifdef HAVE_TRACING
instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
MPI_CURRENT_TYPE = MPI_DEFAULT_TYPE;
MPI_CURRENT_TYPE2 = MPI_DEFAULT_TYPE;
}
- void *sendbuf = get_sendbuffer(sendcount* smpi_datatype_size(MPI_CURRENT_TYPE));
+ void *sendbuf = smpi_get_tmp_sendbuffer(sendcount* smpi_datatype_size(MPI_CURRENT_TYPE));
for(i=0;i<comm_size;i++) {
recvcounts[i] = atoi(action[i+3]);
recv_sum=recv_sum+recvcounts[i];
}
- void *recvbuf = get_recvbuffer(recv_sum* smpi_datatype_size(MPI_CURRENT_TYPE2));
+ void *recvbuf = smpi_get_tmp_recvbuffer(recv_sum* smpi_datatype_size(MPI_CURRENT_TYPE2));
#ifdef HAVE_TRACING
int rank = smpi_process_index();
MPI_CURRENT_TYPE2=MPI_DEFAULT_TYPE;
}
- void *sendbuf = get_sendbuffer(send_buf_size* smpi_datatype_size(MPI_CURRENT_TYPE));
- void *recvbuf = get_recvbuffer(recv_buf_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
+ void *sendbuf = smpi_get_tmp_sendbuffer(send_buf_size* smpi_datatype_size(MPI_CURRENT_TYPE));
+ void *recvbuf = smpi_get_tmp_recvbuffer(recv_buf_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
for(i=0;i<comm_size;i++) {
sendcounts[i] = atoi(action[i+3]);