Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
MPI_Comm -> C++
authordegomme <augustin.degomme@unibas.ch>
Mon, 6 Mar 2017 01:09:30 +0000 (02:09 +0100)
committerdegomme <augustin.degomme@unibas.ch>
Mon, 6 Mar 2017 01:09:30 +0000 (02:09 +0100)
115 files changed:
include/smpi/forward.hpp
include/smpi/smpi.h
src/smpi/colls/allgather-2dmesh.cpp
src/smpi/colls/allgather-3dmesh.cpp
src/smpi/colls/allgather-GB.cpp
src/smpi/colls/allgather-NTSLR-NB.cpp
src/smpi/colls/allgather-NTSLR.cpp
src/smpi/colls/allgather-SMP-NTS.cpp
src/smpi/colls/allgather-bruck.cpp
src/smpi/colls/allgather-loosely-lr.cpp
src/smpi/colls/allgather-mvapich-smp.cpp
src/smpi/colls/allgather-ompi-neighborexchange.cpp
src/smpi/colls/allgather-pair.cpp
src/smpi/colls/allgather-rdb.cpp
src/smpi/colls/allgather-rhv.cpp
src/smpi/colls/allgather-ring.cpp
src/smpi/colls/allgather-smp-simple.cpp
src/smpi/colls/allgather-spreading-simple.cpp
src/smpi/colls/allgatherv-GB.cpp
src/smpi/colls/allgatherv-mpich-rdb.cpp
src/smpi/colls/allgatherv-mpich-ring.cpp
src/smpi/colls/allgatherv-ompi-bruck.cpp
src/smpi/colls/allgatherv-ompi-neighborexchange.cpp
src/smpi/colls/allgatherv-pair.cpp
src/smpi/colls/allgatherv-ring.cpp
src/smpi/colls/allreduce-lr.cpp
src/smpi/colls/allreduce-mvapich-rs.cpp
src/smpi/colls/allreduce-mvapich-two-level.cpp
src/smpi/colls/allreduce-ompi-ring-segmented.cpp
src/smpi/colls/allreduce-rab-rdb.cpp
src/smpi/colls/allreduce-rab1.cpp
src/smpi/colls/allreduce-rab2.cpp
src/smpi/colls/allreduce-rdb.cpp
src/smpi/colls/allreduce-smp-binomial-pipeline.cpp
src/smpi/colls/allreduce-smp-binomial.cpp
src/smpi/colls/allreduce-smp-rdb.cpp
src/smpi/colls/allreduce-smp-rsag-lr.cpp
src/smpi/colls/allreduce-smp-rsag-rab.cpp
src/smpi/colls/allreduce-smp-rsag.cpp
src/smpi/colls/alltoall-2dmesh.cpp
src/smpi/colls/alltoall-3dmesh.cpp
src/smpi/colls/alltoall-bruck.cpp
src/smpi/colls/alltoall-mvapich-scatter-dest.cpp
src/smpi/colls/alltoall-pair-light-barrier.cpp
src/smpi/colls/alltoall-pair-mpi-barrier.cpp
src/smpi/colls/alltoall-pair-one-barrier.cpp
src/smpi/colls/alltoall-pair.cpp
src/smpi/colls/alltoall-rdb.cpp
src/smpi/colls/alltoall-ring-light-barrier.cpp
src/smpi/colls/alltoall-ring-mpi-barrier.cpp
src/smpi/colls/alltoall-ring-one-barrier.cpp
src/smpi/colls/alltoall-ring.cpp
src/smpi/colls/alltoallv-bruck.cpp
src/smpi/colls/alltoallv-ompi-basic-linear.cpp
src/smpi/colls/alltoallv-pair-light-barrier.cpp
src/smpi/colls/alltoallv-pair-mpi-barrier.cpp
src/smpi/colls/alltoallv-pair-one-barrier.cpp
src/smpi/colls/alltoallv-pair.cpp
src/smpi/colls/alltoallv-ring-light-barrier.cpp
src/smpi/colls/alltoallv-ring-mpi-barrier.cpp
src/smpi/colls/alltoallv-ring-one-barrier.cpp
src/smpi/colls/alltoallv-ring.cpp
src/smpi/colls/barrier-mvapich2-pair.cpp
src/smpi/colls/barrier-ompi.cpp
src/smpi/colls/bcast-NTSB.cpp
src/smpi/colls/bcast-NTSL-Isend.cpp
src/smpi/colls/bcast-NTSL.cpp
src/smpi/colls/bcast-SMP-binary.cpp
src/smpi/colls/bcast-SMP-binomial.cpp
src/smpi/colls/bcast-SMP-linear.cpp
src/smpi/colls/bcast-arrival-pattern-aware-wait.cpp
src/smpi/colls/bcast-arrival-pattern-aware.cpp
src/smpi/colls/bcast-arrival-scatter.cpp
src/smpi/colls/bcast-binomial-tree.cpp
src/smpi/colls/bcast-flattree-pipeline.cpp
src/smpi/colls/bcast-flattree.cpp
src/smpi/colls/bcast-mvapich-smp.cpp
src/smpi/colls/bcast-ompi-pipeline.cpp
src/smpi/colls/bcast-ompi-split-bintree.cpp
src/smpi/colls/bcast-scatter-LR-allgather.cpp
src/smpi/colls/bcast-scatter-rdb-allgather.cpp
src/smpi/colls/coll_tuned_topo.cpp
src/smpi/colls/gather-mvapich.cpp
src/smpi/colls/gather-ompi.cpp
src/smpi/colls/reduce-NTSL.cpp
src/smpi/colls/reduce-arrival-pattern-aware.cpp
src/smpi/colls/reduce-binomial.cpp
src/smpi/colls/reduce-flat-tree.cpp
src/smpi/colls/reduce-mvapich-knomial.cpp
src/smpi/colls/reduce-mvapich-two-level.cpp
src/smpi/colls/reduce-ompi.cpp
src/smpi/colls/reduce-scatter-gather.cpp
src/smpi/colls/reduce_scatter-mpich.cpp
src/smpi/colls/reduce_scatter-ompi.cpp
src/smpi/colls/scatter-mvapich-two-level.cpp
src/smpi/colls/scatter-ompi.cpp
src/smpi/colls/smpi_automatic_selector.cpp
src/smpi/colls/smpi_intel_mpi_selector.cpp
src/smpi/colls/smpi_mpich_selector.cpp
src/smpi/colls/smpi_mvapich2_selector.cpp
src/smpi/colls/smpi_openmpi_selector.cpp
src/smpi/private.h
src/smpi/smpi_base.cpp
src/smpi/smpi_bench.cpp
src/smpi/smpi_coll.cpp
src/smpi/smpi_comm.cpp
src/smpi/smpi_comm.hpp [new file with mode: 0644]
src/smpi/smpi_deployment.cpp
src/smpi/smpi_global.cpp
src/smpi/smpi_group.cpp
src/smpi/smpi_pmpi.cpp
src/smpi/smpi_replay.cpp
src/smpi/smpi_rma.cpp
src/smpi/smpi_topo.cpp
tools/cmake/DefinePackages.cmake

index d93e895..e24b1f0 100644 (file)
@@ -14,15 +14,18 @@ namespace simgrid {
 namespace SMPI {
 
 class Group;
 namespace SMPI {
 
 class Group;
+class Comm;
 
 }
 }
 
 typedef simgrid::SMPI::Group SMPI_Group;
 
 }
 }
 
 typedef simgrid::SMPI::Group SMPI_Group;
+typedef simgrid::SMPI::Comm SMPI_Comm;
 
 #else
 
 typedef struct SMPI_Group SMPI_Group;
 
 #else
 
 typedef struct SMPI_Group SMPI_Group;
+typedef struct SMPI_Comm SMPI_Comm;
 
 #endif
 
 
 #endif
 
index bd54ea2..6214488 100644 (file)
@@ -18,7 +18,7 @@
 #include <xbt/function_types.h>
 #include "simgrid/datatypes.h"
 
 #include <xbt/function_types.h>
 #include "simgrid/datatypes.h"
 
-#include "forward.hpp"
+#include "include/smpi/forward.hpp"
 
 #ifdef _WIN32
 #define MPI_CALL(type,name,args) \
 
 #ifdef _WIN32
 #define MPI_CALL(type,name,args) \
@@ -372,8 +372,7 @@ typedef SMPI_Group* MPI_Group;
 
 XBT_PUBLIC_DATA( MPI_Group ) MPI_GROUP_EMPTY;
 
 
 XBT_PUBLIC_DATA( MPI_Group ) MPI_GROUP_EMPTY;
 
-struct s_smpi_mpi_communicator;
-typedef struct s_smpi_mpi_communicator *MPI_Comm;
+typedef SMPI_Comm *MPI_Comm;
 
 #define MPI_COMM_NULL ((MPI_Comm)NULL)
 XBT_PUBLIC_DATA( MPI_Comm ) MPI_COMM_WORLD;
 
 #define MPI_COMM_NULL ((MPI_Comm)NULL)
 XBT_PUBLIC_DATA( MPI_Comm ) MPI_COMM_WORLD;
index 6bec72d..7452654 100644 (file)
@@ -118,8 +118,8 @@ smpi_coll_tuned_allgather_2dmesh(void *send_buff, int send_count, MPI_Datatype
   int my_row_base, my_col_base, src_row_base, block_size, num_reqs;
   int tag = COLL_TAG_ALLGATHER;
 
   int my_row_base, my_col_base, src_row_base, block_size, num_reqs;
   int tag = COLL_TAG_ALLGATHER;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   extent = smpi_datatype_get_extent(send_type);
 
 
   extent = smpi_datatype_get_extent(send_type);
 
index ab964a1..7e57df1 100644 (file)
@@ -105,8 +105,8 @@ int smpi_coll_tuned_allgather_3dmesh(void *send_buff, int send_count,
   int two_dsize, my_row_base, my_col_base, src_row_base, src_z_base, num_reqs;
   int tag = COLL_TAG_ALLGATHER;
 
   int two_dsize, my_row_base, my_col_base, src_row_base, src_z_base, num_reqs;
   int tag = COLL_TAG_ALLGATHER;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   extent = smpi_datatype_get_extent(send_type);
 
   if (!is_3dmesh(num_procs, &X, &Y, &Z))
   extent = smpi_datatype_get_extent(send_type);
 
   if (!is_3dmesh(num_procs, &X, &Y, &Z))
index b1bcc88..054947a 100644 (file)
@@ -13,7 +13,7 @@ int smpi_coll_tuned_allgather_GB(void *send_buff, int send_count,
                                  MPI_Comm comm)
 {
   int num_procs;
                                  MPI_Comm comm)
 {
   int num_procs;
-  num_procs = smpi_comm_size(comm);
+  num_procs = comm->size();
   mpi_coll_gather_fun(send_buff, send_count, send_type, recv_buff, recv_count, recv_type,
              0, comm);
   mpi_coll_bcast_fun(recv_buff, (recv_count * num_procs), recv_type, 0, comm);
   mpi_coll_gather_fun(send_buff, send_count, send_type, recv_buff, recv_count, recv_type,
              0, comm);
   mpi_coll_bcast_fun(recv_buff, (recv_count * num_procs), recv_type, 0, comm);
index 2cabbcd..cfb643f 100644 (file)
@@ -18,8 +18,8 @@ smpi_coll_tuned_allgather_NTSLR_NB(void *sbuf, int scount, MPI_Datatype stype,
   int send_offset, recv_offset;
   int tag = COLL_TAG_ALLGATHER;
 
   int send_offset, recv_offset;
   int tag = COLL_TAG_ALLGATHER;
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
   MPI_Request *rrequest_array;
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
   MPI_Request *rrequest_array;
index 659be11..234ceea 100644 (file)
@@ -18,8 +18,8 @@ smpi_coll_tuned_allgather_NTSLR(void *sbuf, int scount, MPI_Datatype stype,
   int send_offset, recv_offset;
   int tag = COLL_TAG_ALLGATHER;
 
   int send_offset, recv_offset;
   int tag = COLL_TAG_ALLGATHER;
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
 
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
 
index d4838ed..f64ca50 100644 (file)
@@ -12,8 +12,8 @@ int smpi_coll_tuned_allgather_SMP_NTS(void *sbuf, int scount,
                                       MPI_Comm comm)
 {
   int src, dst, comm_size, rank;
                                       MPI_Comm comm)
 {
   int src, dst, comm_size, rank;
-  comm_size = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  comm_size = comm->size();
+  rank = comm->rank();
   MPI_Aint rextent, sextent;
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
   MPI_Aint rextent, sextent;
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
@@ -22,12 +22,12 @@ int smpi_coll_tuned_allgather_SMP_NTS(void *sbuf, int scount,
   int i, send_offset, recv_offset;
   int intra_rank, inter_rank;
 
   int i, send_offset, recv_offset;
   int intra_rank, inter_rank;
 
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
 
 
   }
 
 
index e0424e1..aaab5e3 100644 (file)
@@ -87,8 +87,8 @@ int smpi_coll_tuned_allgather_bruck(void *send_buff, int send_count,
   char *recv_ptr = (char *) recv_buff;
 
   // get size of the communicator, followed by rank 
   char *recv_ptr = (char *) recv_buff;
 
   // get size of the communicator, followed by rank 
-  num_procs = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  num_procs = comm->size();
+  rank = comm->rank();
 
   // get size of single element's type for recv buffer
   recv_extent = smpi_datatype_get_extent(recv_type);
 
   // get size of single element's type for recv buffer
   recv_extent = smpi_datatype_get_extent(recv_type);
index 74df884..8724dda 100644 (file)
@@ -17,20 +17,20 @@ int smpi_coll_tuned_allgather_loosely_lr(void *sbuf, int scount,
   int intra_rank, inter_rank, inter_comm_size, intra_comm_size;
   int inter_dst, inter_src;
 
   int intra_rank, inter_rank, inter_comm_size, intra_comm_size;
   int inter_dst, inter_src;
 
-  comm_size = smpi_comm_size(comm);
+  comm_size = comm->size();
 
 
-if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
 
   if(comm_size%num_core)
     THROWF(arg_error,0, "allgather loosely lr algorithm can't be used with non multiple of NUM_CORE=%d number of processes ! ",num_core);
 
   }
 
   if(comm_size%num_core)
     THROWF(arg_error,0, "allgather loosely lr algorithm can't be used with non multiple of NUM_CORE=%d number of processes ! ",num_core);
 
-  rank = smpi_comm_rank(comm);
+  rank = comm->rank();
   MPI_Aint rextent, sextent;
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
   MPI_Aint rextent, sextent;
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
index e479361..5e79dea 100644 (file)
@@ -49,34 +49,34 @@ int smpi_coll_tuned_allgather_mvapich2_smp(void *sendbuf,int sendcnt, MPI_Dataty
     MPI_Aint recvtype_extent = 0;  /* Datatype extent */
     MPI_Comm shmem_comm, leader_comm;
 
     MPI_Aint recvtype_extent = 0;  /* Datatype extent */
     MPI_Comm shmem_comm, leader_comm;
 
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   
   }
   
-    if(!smpi_comm_is_uniform(comm) || !smpi_comm_is_blocked(comm))
+    if(!comm->is_uniform() || !comm->is_blocked())
     THROWF(arg_error,0, "allgather MVAPICH2 smp algorithm can't be used with irregular deployment. Please insure that processes deployed on the same node are contiguous and that each node has the same number of processes");
   
     if (recvcnt == 0) {
         return MPI_SUCCESS;
     }
 
     THROWF(arg_error,0, "allgather MVAPICH2 smp algorithm can't be used with irregular deployment. Please insure that processes deployed on the same node are contiguous and that each node has the same number of processes");
   
     if (recvcnt == 0) {
         return MPI_SUCCESS;
     }
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
 
     /* extract the rank,size information for the intra-node communicator */
     recvtype_extent=smpi_datatype_get_extent(recvtype);
     
 
     /* extract the rank,size information for the intra-node communicator */
     recvtype_extent=smpi_datatype_get_extent(recvtype);
     
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_rank = smpi_comm_rank(shmem_comm);
-    local_size = smpi_comm_size(shmem_comm);
+    shmem_comm = comm->get_intra_comm();
+    local_rank = shmem_comm->rank();
+    local_size = shmem_comm->size();
 
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader communicator */
 
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader communicator */
-        leader_comm = smpi_comm_get_leaders_comm(comm);
+        leader_comm = comm->get_leaders_comm();
         if(leader_comm==MPI_COMM_NULL){
           leader_comm = MPI_COMM_WORLD;
         }
         if(leader_comm==MPI_COMM_NULL){
           leader_comm = MPI_COMM_WORLD;
         }
-        leader_comm_size = smpi_comm_size(leader_comm);
+        leader_comm_size = leader_comm->size();
     }
 
     /*If there is just one node, after gather itself,
     }
 
     /*If there is just one node, after gather itself,
@@ -103,14 +103,14 @@ int smpi_coll_tuned_allgather_mvapich2_smp(void *sendbuf,int sendcnt, MPI_Dataty
     /* Exchange the data between the node leaders*/
     if (local_rank == 0 && (leader_comm_size > 1)) {
         /*When data in each socket is different*/
     /* Exchange the data between the node leaders*/
     if (local_rank == 0 && (leader_comm_size > 1)) {
         /*When data in each socket is different*/
-        if (smpi_comm_is_uniform(comm) != 1) {
+        if (comm->is_uniform() != 1) {
 
             int *displs = NULL;
             int *recvcnts = NULL;
             int *node_sizes = NULL;
             int i = 0;
 
 
             int *displs = NULL;
             int *recvcnts = NULL;
             int *node_sizes = NULL;
             int i = 0;
 
-            node_sizes = smpi_comm_get_non_uniform_map(comm);
+            node_sizes = comm->get_non_uniform_map();
 
             displs =  static_cast<int *>(xbt_malloc(sizeof (int) * leader_comm_size));
             recvcnts =  static_cast<int *>(xbt_malloc(sizeof (int) * leader_comm_size));
 
             displs =  static_cast<int *>(xbt_malloc(sizeof (int) * leader_comm_size));
             recvcnts =  static_cast<int *>(xbt_malloc(sizeof (int) * leader_comm_size));
@@ -126,7 +126,7 @@ int smpi_coll_tuned_allgather_mvapich2_smp(void *sendbuf,int sendcnt, MPI_Dataty
             }
 
 
             }
 
 
-            void* sendbuf=((char*)recvbuf)+smpi_datatype_get_extent(recvtype)*displs[smpi_comm_rank(leader_comm)];
+            void* sendbuf=((char*)recvbuf)+smpi_datatype_get_extent(recvtype)*displs[leader_comm->rank()];
 
             mpi_errno = mpi_coll_allgatherv_fun(sendbuf,
                                        (recvcnt*local_size),
 
             mpi_errno = mpi_coll_allgatherv_fun(sendbuf,
                                        (recvcnt*local_size),
@@ -137,7 +137,7 @@ int smpi_coll_tuned_allgather_mvapich2_smp(void *sendbuf,int sendcnt, MPI_Dataty
             xbt_free(displs);
             xbt_free(recvcnts);
         } else {
             xbt_free(displs);
             xbt_free(recvcnts);
         } else {
-        void* sendtmpbuf=((char*)recvbuf)+smpi_datatype_get_extent(recvtype)*(recvcnt*local_size)*smpi_comm_rank(leader_comm);
+        void* sendtmpbuf=((char*)recvbuf)+smpi_datatype_get_extent(recvtype)*(recvcnt*local_size)*leader_comm->rank();
         
           
 
         
           
 
index 9a3ccd9..9227d51 100644 (file)
@@ -79,8 +79,8 @@ smpi_coll_tuned_allgather_ompi_neighborexchange(void *sbuf, int scount,
    ptrdiff_t slb, rlb, sext, rext;
    char *tmpsend = NULL, *tmprecv = NULL;
 
    ptrdiff_t slb, rlb, sext, rext;
    char *tmpsend = NULL, *tmprecv = NULL;
 
-   size = smpi_comm_size(comm);
-   rank = smpi_comm_rank(comm);
+   size = comm->size();
+   rank = comm->rank();
 
    if (size % 2) {
       XBT_DEBUG(
 
    if (size % 2) {
       XBT_DEBUG(
index 6111f28..1d1d2b0 100644 (file)
@@ -79,8 +79,8 @@ smpi_coll_tuned_allgather_pair(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  unsigned int rank = smpi_comm_rank(comm);
-  unsigned int num_procs = smpi_comm_size(comm);
+  unsigned int rank = comm->rank();
+  unsigned int num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "allgather pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "allgather pair algorithm can't be used with non power of two number of processes ! ");
index 745af64..1d43ca2 100644 (file)
@@ -30,8 +30,8 @@ smpi_coll_tuned_allgather_rdb(void *sbuf, int send_count,
   char *recv_ptr = (char *) rbuf;
 
   // get size of the communicator, followed by rank 
   char *recv_ptr = (char *) rbuf;
 
   // get size of the communicator, followed by rank 
-  unsigned int num_procs = smpi_comm_size(comm);
-  unsigned int rank = smpi_comm_rank(comm);
+  unsigned int num_procs = comm->size();
+  unsigned int rank = comm->rank();
 
   // get size of single element's type for send buffer and recv buffer
   send_chunk = smpi_datatype_get_extent(send_type);
 
   // get size of single element's type for send buffer and recv buffer
   send_chunk = smpi_datatype_get_extent(send_type);
index 6b41f55..08edb89 100644 (file)
@@ -25,12 +25,12 @@ smpi_coll_tuned_allgather_rhv(void *sbuf, int send_count,
   int curr_count;
 
   // get size of the communicator, followed by rank 
   int curr_count;
 
   // get size of the communicator, followed by rank 
-  unsigned int num_procs = smpi_comm_size(comm);
+  unsigned int num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "allgather rhv algorithm can't be used with non power of two number of processes ! ");
 
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "allgather rhv algorithm can't be used with non power of two number of processes ! ");
 
-  unsigned int rank = smpi_comm_rank(comm);
+  unsigned int rank = comm->rank();
 
   // get size of single element's type for send buffer and recv buffer
   s_extent = smpi_datatype_get_extent(send_type);
 
   // get size of single element's type for send buffer and recv buffer
   s_extent = smpi_datatype_get_extent(send_type);
index 0e96a53..2a6d84a 100644 (file)
@@ -78,8 +78,8 @@ smpi_coll_tuned_allgather_ring(void *send_buff, int send_count,
   char *sendptr = (char *) send_buff;
   char *recvptr = (char *) recv_buff;
 
   char *sendptr = (char *) send_buff;
   char *recvptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   extent = smpi_datatype_get_extent(send_type);
 
   // local send/recv
   extent = smpi_datatype_get_extent(send_type);
 
   // local send/recv
index e54c75b..1db7bc2 100644 (file)
@@ -12,20 +12,20 @@ int smpi_coll_tuned_allgather_smp_simple(void *send_buf, int scount,
                                          MPI_Comm comm)
 {
   int src, dst, comm_size, rank;
                                          MPI_Comm comm)
 {
   int src, dst, comm_size, rank;
-  comm_size = smpi_comm_size(comm);
+  comm_size = comm->size();
 
 
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
 
   if(comm_size%num_core)
      THROWF(arg_error,0, "allgather SMP simple algorithm can't be used with non multiple of NUM_CORE=%d number of processes ! ", num_core);
 
   }
 
   if(comm_size%num_core)
      THROWF(arg_error,0, "allgather SMP simple algorithm can't be used with non multiple of NUM_CORE=%d number of processes ! ", num_core);
 
-  rank = smpi_comm_rank(comm);
+  rank = comm->rank();
   MPI_Aint rextent, sextent;
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
   MPI_Aint rextent, sextent;
   rextent = smpi_datatype_get_extent(rtype);
   sextent = smpi_datatype_get_extent(stype);
index 1f51ea4..671f9fc 100644 (file)
@@ -80,8 +80,8 @@ smpi_coll_tuned_allgather_spreading_simple(void *send_buff, int send_count,
   MPI_Status status;
   char *recv_ptr = (char *) recv_buff;
 
   MPI_Status status;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   extent = smpi_datatype_get_extent(send_type);
 
   num_reqs = (2 * num_procs) - 2;
   extent = smpi_datatype_get_extent(send_type);
 
   num_reqs = (2 * num_procs) - 2;
index 1dd6103..7941bca 100644 (file)
@@ -15,7 +15,7 @@ int smpi_coll_tuned_allgatherv_GB(void *send_buff, int send_count,
   smpi_mpi_gatherv(send_buff, send_count, send_type, recv_buff, recv_counts,
                   recv_disps, recv_type, 0, comm);
   int num_procs, i, current, max = 0;
   smpi_mpi_gatherv(send_buff, send_count, send_type, recv_buff, recv_counts,
                   recv_disps, recv_type, 0, comm);
   int num_procs, i, current, max = 0;
-  num_procs = smpi_comm_size(comm);
+  num_procs = comm->size();
   for (i = 0; i < num_procs; i++) {
     current = recv_disps[i] + recv_counts[i];
     if (current > max)
   for (i = 0; i < num_procs; i++) {
     current = recv_disps[i] + recv_counts[i];
     if (current > max)
index 099297c..b2ccc5d 100644 (file)
@@ -26,8 +26,8 @@ int smpi_coll_tuned_allgatherv_mpich_rdb (
     send_offset, recv_offset, last_recv_cnt=0, nprocs_completed, k,
     offset, tmp_mask, tree_root;
 
     send_offset, recv_offset, last_recv_cnt=0, nprocs_completed, k,
     offset, tmp_mask, tree_root;
 
-  unsigned int comm_size = smpi_comm_size(comm);
-  unsigned int rank = smpi_comm_rank(comm);
+  unsigned int comm_size = comm->size();
+  unsigned int rank = comm->rank();
 
   total_count = 0;
   for (i=0; i<comm_size; i++)
 
   total_count = 0;
   for (i=0; i<comm_size; i++)
index fc72645..c128ba4 100644 (file)
@@ -39,8 +39,8 @@ smpi_coll_tuned_allgatherv_mpich_ring(void *sendbuf, int sendcount,
   MPI_Status status;
   MPI_Aint recvtype_extent;
   int right, left, total_count, i;
   MPI_Status status;
   MPI_Aint recvtype_extent;
   int right, left, total_count, i;
-  rank= smpi_comm_rank(comm);
-  comm_size=smpi_comm_size(comm);
+  rank= comm->rank();
+  comm_size=comm->size();
 
   recvtype_extent= smpi_datatype_get_extent( recvtype);
   total_count = 0;
 
   recvtype_extent= smpi_datatype_get_extent( recvtype);
   total_count = 0;
index d87dbab..d41691c 100644 (file)
@@ -91,8 +91,8 @@ int smpi_coll_tuned_allgatherv_ompi_bruck(void *sbuf, int scount,
    char *tmpsend = NULL, *tmprecv = NULL;
    MPI_Datatype new_rdtype = MPI_DATATYPE_NULL, new_sdtype = MPI_DATATYPE_NULL;
 
    char *tmpsend = NULL, *tmprecv = NULL;
    MPI_Datatype new_rdtype = MPI_DATATYPE_NULL, new_sdtype = MPI_DATATYPE_NULL;
 
-   unsigned int size = smpi_comm_size(comm);
-   unsigned int rank = smpi_comm_rank(comm);
+   unsigned int size = comm->size();
+   unsigned int rank = comm->rank();
 
    XBT_DEBUG(
                 "coll:tuned:allgather_ompi_bruck rank %d", rank);
 
    XBT_DEBUG(
                 "coll:tuned:allgather_ompi_bruck rank %d", rank);
index edf93c3..7392c4b 100644 (file)
@@ -82,8 +82,8 @@ smpi_coll_tuned_allgatherv_ompi_neighborexchange(void *sbuf, int scount,
     char *tmpsend = NULL, *tmprecv = NULL;
 
 
     char *tmpsend = NULL, *tmprecv = NULL;
 
 
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     if (size % 2) {
         XBT_DEBUG(
 
     if (size % 2) {
         XBT_DEBUG(
index b6eea73..e5a5ac6 100644 (file)
@@ -79,8 +79,8 @@ smpi_coll_tuned_allgatherv_pair(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  unsigned int rank = smpi_comm_rank(comm);
-  unsigned int num_procs = smpi_comm_size(comm);
+  unsigned int rank = comm->rank();
+  unsigned int num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "allgatherv pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "allgatherv pair algorithm can't be used with non power of two number of processes ! ");
index 8ac68b7..b21f652 100644 (file)
@@ -78,8 +78,8 @@ smpi_coll_tuned_allgatherv_ring(void *send_buff, int send_count,
   char *sendptr = (char *) send_buff;
   char *recvptr = (char *) recv_buff;
 
   char *sendptr = (char *) send_buff;
   char *recvptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   extent = smpi_datatype_get_extent(send_type);
 
   // local send/recv
   extent = smpi_datatype_get_extent(send_type);
 
   // local send/recv
index 9bcf75c..61ef576 100644 (file)
@@ -29,8 +29,8 @@ smpi_coll_tuned_allreduce_lr(void *sbuf, void *rbuf, int rcount,
   int send_offset, recv_offset;
   int remainder, remainder_flag, remainder_offset;
 
   int send_offset, recv_offset;
   int remainder, remainder_flag, remainder_offset;
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
   /* make it compatible with all data type */
   MPI_Aint extent;
 
   /* make it compatible with all data type */
   MPI_Aint extent;
index 10940e3..177b545 100644 (file)
@@ -43,8 +43,8 @@ int smpi_coll_tuned_allreduce_mvapich2_rs(void *sendbuf,
 
     /* homogeneous */
 
 
     /* homogeneous */
 
-    int comm_size =  smpi_comm_size(comm);
-    int rank = smpi_comm_rank(comm);
+    int comm_size =  comm->size();
+    int rank = comm->rank();
 
     is_commutative = smpi_op_is_commute(op);
 
 
     is_commutative = smpi_op_is_commute(op);
 
index 90ced9a..def7b4f 100644 (file)
@@ -93,8 +93,8 @@ int smpi_coll_tuned_allreduce_mvapich2_two_level(void *sendbuf,
     if(MV2_Allreduce_function==NULL)
       MV2_Allreduce_function = smpi_coll_tuned_allreduce_rdb;
     
     if(MV2_Allreduce_function==NULL)
       MV2_Allreduce_function = smpi_coll_tuned_allreduce_rdb;
     
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
   
     if (count == 0) {
     }
   
     if (count == 0) {
@@ -103,12 +103,12 @@ int smpi_coll_tuned_allreduce_mvapich2_two_level(void *sendbuf,
     smpi_datatype_extent(datatype, &true_lb,
                                        &true_extent);
 
     smpi_datatype_extent(datatype, &true_lb,
                                        &true_extent);
 
-    total_size = smpi_comm_size(comm);
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_rank = smpi_comm_rank(shmem_comm);
-    local_size = smpi_comm_size(shmem_comm);
+    total_size = comm->size();
+    shmem_comm = comm->get_intra_comm();
+    local_rank = shmem_comm->rank();
+    local_size = shmem_comm->size();
 
 
-    leader_comm = smpi_comm_get_leaders_comm(comm);
+    leader_comm = comm->get_leaders_comm();
 
     if (local_rank == 0) {
         if (sendbuf != MPI_IN_PLACE) {
 
     if (local_rank == 0) {
         if (sendbuf != MPI_IN_PLACE) {
index 6c85ced..cbe09f9 100644 (file)
@@ -175,8 +175,8 @@ smpi_coll_tuned_allreduce_ompi_ring_segmented(void *sbuf, void *rbuf, int count,
    ptrdiff_t block_offset, max_real_segsize;
    MPI_Request reqs[2] = {NULL, NULL};
    const size_t segsize = 1 << 20; /* 1 MB */
    ptrdiff_t block_offset, max_real_segsize;
    MPI_Request reqs[2] = {NULL, NULL};
    const size_t segsize = 1 << 20; /* 1 MB */
-   int size = smpi_comm_size(comm);
-   int rank = smpi_comm_rank(comm);
+   int size = comm->size();
+   int rank = comm->rank();
 
    XBT_DEBUG("coll:tuned:allreduce_intra_ring_segmented rank %d, count %d", rank, count);
 
 
    XBT_DEBUG("coll:tuned:allreduce_intra_ring_segmented rank %d, count %d", rank, count);
 
index 27f3628..a4de224 100644 (file)
@@ -18,8 +18,8 @@ int smpi_coll_tuned_allreduce_rab_rdb(void *sbuff, void *rbuff, int count,
   MPI_Status status;
   void *tmp_buf = NULL;
 
   MPI_Status status;
   void *tmp_buf = NULL;
 
-  unsigned int nprocs = smpi_comm_size(comm);
-  int rank = smpi_comm_rank(comm);
+  unsigned int nprocs = comm->size();
+  int rank = comm->rank();
 
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
 
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index cb31de6..eaf238d 100644 (file)
@@ -20,8 +20,8 @@ int smpi_coll_tuned_allreduce_rab1(void *sbuff, void *rbuff,
 
   void *recv, *tmp_buf;
 
 
   void *recv, *tmp_buf;
 
-  int rank = smpi_comm_rank(comm);
-  unsigned int nprocs = smpi_comm_size(comm);
+  int rank = comm->rank();
+  unsigned int nprocs = comm->size();
 
   if((nprocs&(nprocs-1)))
     THROWF(arg_error,0, "allreduce rab1 algorithm can't be used with non power of two number of processes ! ");
 
   if((nprocs&(nprocs-1)))
     THROWF(arg_error,0, "allreduce rab1 algorithm can't be used with non power of two number of processes ! ");
index 372d5e8..02f4995 100644 (file)
@@ -26,8 +26,8 @@ int smpi_coll_tuned_allreduce_rab2(void *sbuff, void *rbuff,
      uop  = op_ptr->op;
      #endif
    */
      uop  = op_ptr->op;
      #endif
    */
-  rank = smpi_comm_rank(comm);
-  nprocs = smpi_comm_size(comm);
+  rank = comm->rank();
+  nprocs = comm->size();
 
 
   s_extent = smpi_datatype_get_extent(dtype);
 
 
   s_extent = smpi_datatype_get_extent(dtype);
index 85a31ef..450b596 100644 (file)
@@ -25,8 +25,8 @@ int smpi_coll_tuned_allreduce_rdb(void *sbuff, void *rbuff, int count,
      uop  = op_ptr->op;
      #endif
    */
      uop  = op_ptr->op;
      #endif
    */
-  nprocs=smpi_comm_size(comm);
-  rank=smpi_comm_rank(comm);
+  nprocs=comm->size();
+  rank=comm->rank();
 
   smpi_datatype_extent(dtype, &lb, &extent);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
 
   smpi_datatype_extent(dtype, &lb, &extent);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index 06a80a7..5af338c 100644 (file)
@@ -48,16 +48,16 @@ int smpi_coll_tuned_allreduce_smp_binomial_pipeline(void *send_buf,
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
 
   }
 
-  comm_size = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  comm_size = comm->size();
+  rank = comm->rank();
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index 3d159be..3782993 100644 (file)
@@ -36,17 +36,17 @@ int smpi_coll_tuned_allreduce_smp_binomial(void *send_buf, void *recv_buf,
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
 
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
 
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
   MPI_Status status;
 
   }
   MPI_Status status;
 
-  comm_size=smpi_comm_size(comm);
-  rank=smpi_comm_rank(comm);
+  comm_size=comm->size();
+  rank=comm->rank();
   MPI_Aint extent, lb;
   smpi_datatype_extent(dtype, &lb, &extent);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
   MPI_Aint extent, lb;
   smpi_datatype_extent(dtype, &lb, &extent);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index 9770d29..b95bf7f 100644 (file)
@@ -35,12 +35,12 @@ int smpi_coll_tuned_allreduce_smp_rdb(void *send_buf, void *recv_buf, int count,
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
   /*
      #ifdef MPICH2_REDUCTION
   }
   /*
      #ifdef MPICH2_REDUCTION
@@ -52,8 +52,8 @@ int smpi_coll_tuned_allreduce_smp_rdb(void *send_buf, void *recv_buf, int count,
      uop  = op_ptr->op;
      #endif
    */
      uop  = op_ptr->op;
      #endif
    */
-  comm_size = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  comm_size = comm->size();
+  rank = comm->rank();
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index fa9296f..7e3dfef 100644 (file)
@@ -23,12 +23,12 @@ int smpi_coll_tuned_allreduce_smp_rsag_lr(void *send_buf, void *recv_buf,
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
   /*
      #ifdef MPICH2_REDUCTION
   }
   /*
      #ifdef MPICH2_REDUCTION
@@ -40,8 +40,8 @@ int smpi_coll_tuned_allreduce_smp_rsag_lr(void *send_buf, void *recv_buf,
      uop  = op_ptr->op;
      #endif
    */
      uop  = op_ptr->op;
      #endif
    */
-  comm_size = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  comm_size = comm->size();
+  rank = comm->rank();
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index 1ac827e..64c1729 100644 (file)
@@ -27,20 +27,20 @@ int smpi_coll_tuned_allreduce_smp_rsag_rab(void *sbuf, void *rbuf, int count,
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
 
   }
 
-  comm_size = smpi_comm_size(comm);
+  comm_size = comm->size();
 
   if((comm_size&(comm_size-1)))
     THROWF(arg_error,0, "allreduce smp rsag rab algorithm can't be used with non power of two number of processes ! ");
 
 
   if((comm_size&(comm_size-1)))
     THROWF(arg_error,0, "allreduce smp rsag rab algorithm can't be used with non power of two number of processes ! ");
 
-  rank = smpi_comm_rank(comm);
+  rank = comm->rank();
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index b4a6885..298c978 100644 (file)
@@ -22,12 +22,12 @@ int smpi_coll_tuned_allreduce_smp_rsag(void *send_buf, void *recv_buf,
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
   int tag = COLL_TAG_ALLREDUCE;
   int mask, src, dst;
   MPI_Status status;
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }
   /*
      #ifdef MPICH2_REDUCTION
   }
   /*
      #ifdef MPICH2_REDUCTION
@@ -39,8 +39,8 @@ int smpi_coll_tuned_allreduce_smp_rsag(void *send_buf, void *recv_buf,
      uop  = op_ptr->op;
      #endif
    */
      uop  = op_ptr->op;
      #endif
    */
-  comm_size = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  comm_size = comm->size();
+  rank = comm->rank();
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(dtype);
   tmp_buf = (void *) smpi_get_tmp_sendbuffer(count * extent);
index dc99716..784552a 100644 (file)
@@ -68,8 +68,8 @@ int smpi_coll_tuned_alltoall_2dmesh(void *send_buff, int send_count,
   int my_row_base, my_col_base, src_row_base, block_size;
   int tag = COLL_TAG_ALLTOALL;
 
   int my_row_base, my_col_base, src_row_base, block_size;
   int tag = COLL_TAG_ALLTOALL;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   extent = smpi_datatype_get_extent(send_type);
 
   if (!alltoall_check_is_2dmesh(num_procs, &X, &Y))
   extent = smpi_datatype_get_extent(send_type);
 
   if (!alltoall_check_is_2dmesh(num_procs, &X, &Y))
index 81b1ecd..2c4b10c 100644 (file)
@@ -60,8 +60,8 @@ int smpi_coll_tuned_alltoall_3dmesh(void *send_buff, int send_count,
 
   char *tmp_buff1, *tmp_buff2;
 
 
   char *tmp_buff1, *tmp_buff2;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   extent = smpi_datatype_get_extent(send_type);
 
   if (!alltoall_check_is_3dmesh(num_procs, &X, &Y, &Z))
   extent = smpi_datatype_get_extent(send_type);
 
   if (!alltoall_check_is_3dmesh(num_procs, &X, &Y, &Z))
index 10b4d9c..903931b 100644 (file)
@@ -43,8 +43,8 @@ smpi_coll_tuned_alltoall_bruck(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  num_procs = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  num_procs = comm->size();
+  rank = comm->rank();
 
   extent = smpi_datatype_get_extent(recv_type);
 
 
   extent = smpi_datatype_get_extent(recv_type);
 
index 4194697..92b8066 100644 (file)
@@ -61,8 +61,8 @@ int smpi_coll_tuned_alltoall_mvapich2_scatter_dest(
     
     if (recvcount == 0) return MPI_SUCCESS;
     
     
     if (recvcount == 0) return MPI_SUCCESS;
     
-    comm_size =  smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size =  comm->size();
+    rank = comm->rank();
     
     /* Get extent of send and recv types */
     recvtype_extent = smpi_datatype_get_extent(recvtype);
     
     /* Get extent of send and recv types */
     recvtype_extent = smpi_datatype_get_extent(recvtype);
index 0232b51..448fa95 100644 (file)
@@ -43,8 +43,8 @@ smpi_coll_tuned_alltoall_pair_light_barrier(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
index 42823b5..a324590 100644 (file)
@@ -40,8 +40,8 @@ smpi_coll_tuned_alltoall_pair_mpi_barrier(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
index a0ac128..04c9b85 100644 (file)
@@ -41,8 +41,8 @@ smpi_coll_tuned_alltoall_pair_one_barrier(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
index 8a0a76e..a7731c1 100644 (file)
@@ -40,8 +40,8 @@ int smpi_coll_tuned_alltoall_pair_rma(void *send_buff, int send_count, MPI_Datat
 
   char *send_ptr = (char *) send_buff;
 
 
   char *send_ptr = (char *) send_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
@@ -75,8 +75,8 @@ int smpi_coll_tuned_alltoall_pair(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoall pair algorithm can't be used with non power of two number of processes ! ");
index cb49bfc..ff417b3 100644 (file)
@@ -45,8 +45,8 @@ int smpi_coll_tuned_alltoall_rdb(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  num_procs = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  num_procs = comm->size();
+  rank = comm->rank();
   send_increment = smpi_datatype_get_extent(send_type);
   recv_increment = smpi_datatype_get_extent(recv_type);
   extent = smpi_datatype_get_extent(recv_type);
   send_increment = smpi_datatype_get_extent(send_type);
   recv_increment = smpi_datatype_get_extent(recv_type);
   extent = smpi_datatype_get_extent(recv_type);
index eb3de84..26e91c2 100644 (file)
@@ -43,8 +43,8 @@ smpi_coll_tuned_alltoall_ring_light_barrier(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
index 970a265..2c8a9dc 100644 (file)
@@ -40,8 +40,8 @@ smpi_coll_tuned_alltoall_ring_mpi_barrier(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
index 49f7802..ca0a770 100644 (file)
@@ -39,8 +39,8 @@ smpi_coll_tuned_alltoall_ring_one_barrier(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
index af1cdbf..b871173 100644 (file)
@@ -39,8 +39,8 @@ smpi_coll_tuned_alltoall_ring(void *send_buff, int send_count,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
index d7d0c37..9d18f95 100644 (file)
@@ -26,8 +26,8 @@ int smpi_coll_tuned_alltoallv_bruck(void *sendbuf, int *sendcounts, int *senddis
   MPI_Request *requests;
 
   // FIXME: check implementation
   MPI_Request *requests;
 
   // FIXME: check implementation
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
   XBT_DEBUG("<%d> algorithm alltoall_bruck() called.", rank);
 
   smpi_datatype_extent(sendtype, &lb, &sendext);
   XBT_DEBUG("<%d> algorithm alltoall_bruck() called.", rank);
 
   smpi_datatype_extent(sendtype, &lb, &sendext);
index 6b170be..0f55bda 100644 (file)
@@ -26,8 +26,8 @@ smpi_coll_tuned_alltoallv_ompi_basic_linear(void *sbuf, int *scounts, int *sdisp
     int nreqs;
     ptrdiff_t sext, rext;
     MPI_Request *preq;
     int nreqs;
     ptrdiff_t sext, rext;
     MPI_Request *preq;
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
     MPI_Request *ireqs= static_cast<MPI_Request*>(xbt_malloc(sizeof(MPI_Request) * size * 2));
     XBT_DEBUG(
                  "coll:tuned:alltoallv_intra_basic_linear rank %d", rank);
     MPI_Request *ireqs= static_cast<MPI_Request*>(xbt_malloc(sizeof(MPI_Request) * size * 2));
     XBT_DEBUG(
                  "coll:tuned:alltoallv_intra_basic_linear rank %d", rank);
index 20deb83..ea05e33 100644 (file)
@@ -43,8 +43,8 @@ smpi_coll_tuned_alltoallv_pair_light_barrier(void *send_buff, int *send_counts,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
index b5b05e8..95231a1 100644 (file)
@@ -40,8 +40,8 @@ smpi_coll_tuned_alltoallv_pair_mpi_barrier(void *send_buff, int *send_counts, in
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
index 1039812..c53ddde 100644 (file)
@@ -40,8 +40,8 @@ smpi_coll_tuned_alltoallv_pair_one_barrier(void *send_buff, int *send_counts, in
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
index 43940e7..cb2551d 100644 (file)
@@ -40,8 +40,8 @@ int smpi_coll_tuned_alltoallv_pair(void *send_buff, int *send_counts, int *send_
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
 
   if((num_procs&(num_procs-1)))
     THROWF(arg_error,0, "alltoallv pair algorithm can't be used with non power of two number of processes ! ");
index b493dd7..0e3086f 100644 (file)
@@ -43,8 +43,8 @@ smpi_coll_tuned_alltoallv_ring_light_barrier(void *send_buff, int *send_counts,
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
index 723354b..ed7030f 100644 (file)
@@ -40,8 +40,8 @@ smpi_coll_tuned_alltoallv_ring_mpi_barrier(void *send_buff, int *send_counts, in
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
index e4fdbc4..443bebe 100644 (file)
@@ -39,8 +39,8 @@ smpi_coll_tuned_alltoallv_ring_one_barrier(void *send_buff, int *send_counts, in
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
 
index 3f2ced2..cc46d4c 100644 (file)
@@ -40,8 +40,8 @@ smpi_coll_tuned_alltoallv_ring(void *send_buff, int *send_counts, int *send_disp
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
   char *send_ptr = (char *) send_buff;
   char *recv_ptr = (char *) recv_buff;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
   int pof2 = ((num_procs != 0) && ((num_procs & (~num_procs + 1)) == num_procs));
   send_chunk = smpi_datatype_get_extent(send_type);
   recv_chunk = smpi_datatype_get_extent(recv_type);
   int pof2 = ((num_procs != 0) && ((num_procs & (~num_procs + 1)) == num_procs));
index 2ef6ad1..9683162 100644 (file)
@@ -49,12 +49,12 @@ int smpi_coll_tuned_barrier_mvapich2_pair(MPI_Comm comm)
     int d, dst, src;
     int mpi_errno = MPI_SUCCESS;
 
     int d, dst, src;
     int mpi_errno = MPI_SUCCESS;
 
-    size = smpi_comm_size(comm);
+    size = comm->size();
     /* Trivial barriers return immediately */
     if (size == 1)
         return MPI_SUCCESS;
 
     /* Trivial barriers return immediately */
     if (size == 1)
         return MPI_SUCCESS;
 
-    rank =  smpi_comm_rank(comm);
+    rank =  comm->rank();
     int N2_prev = 1;
     /*  N2_prev = greatest power of two < size of Comm  */
     for( N2_prev = 1; N2_prev <= size; N2_prev <<= 1 );
     int N2_prev = 1;
     /*  N2_prev = greatest power of two < size of Comm  */
     for( N2_prev = 1; N2_prev <= size; N2_prev <<= 1 );
index d61177d..3d9f943 100644 (file)
@@ -50,8 +50,8 @@ int smpi_coll_tuned_barrier_ompi_doublering(MPI_Comm comm
     int left, right;
 
 
     int left, right;
 
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
 
     XBT_DEBUG("ompi_coll_tuned_barrier_ompi_doublering rank %d", rank);
 
 
     XBT_DEBUG("ompi_coll_tuned_barrier_ompi_doublering rank %d", rank);
 
@@ -110,8 +110,8 @@ int smpi_coll_tuned_barrier_ompi_recursivedoubling(MPI_Comm comm
     int rank, size, adjsize;
     int mask, remote;
 
     int rank, size, adjsize;
     int mask, remote;
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_recursivedoubling rank %d", 
                  rank);
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_recursivedoubling rank %d", 
                  rank);
@@ -185,8 +185,8 @@ int smpi_coll_tuned_barrier_ompi_bruck(MPI_Comm comm
     int rank, size;
     int distance, to, from;
 
     int rank, size;
     int distance, to, from;
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_bruck rank %d", rank);
 
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_bruck rank %d", rank);
 
@@ -217,7 +217,7 @@ int smpi_coll_tuned_barrier_ompi_two_procs(MPI_Comm comm
 {
     int remote;
 
 {
     int remote;
 
-    remote = smpi_comm_rank(comm);
+    remote = comm->rank();
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_two_procs rank %d", remote);
     remote = (remote + 1) & 0x1;
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_two_procs rank %d", remote);
     remote = (remote + 1) & 0x1;
@@ -248,8 +248,8 @@ int smpi_coll_tuned_barrier_ompi_two_procs(MPI_Comm comm
 int smpi_coll_tuned_barrier_ompi_basic_linear(MPI_Comm comm)
 {
     int i;
 int smpi_coll_tuned_barrier_ompi_basic_linear(MPI_Comm comm)
 {
     int i;
-    int size = smpi_comm_size(comm);
-    int rank = smpi_comm_rank(comm);
+    int size = comm->size();
+    int rank = comm->rank();
 
     /* All non-root send & receive zero-length message. */
 
 
     /* All non-root send & receive zero-length message. */
 
@@ -302,8 +302,8 @@ int smpi_coll_tuned_barrier_ompi_tree(MPI_Comm comm)
     int rank, size, depth;
     int jump, partner;
 
     int rank, size, depth;
     int jump, partner;
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_tree %d", 
                  rank);
     XBT_DEBUG(
                  "ompi_coll_tuned_barrier_ompi_tree %d", 
                  rank);
index ffdcfc2..5869f15 100644 (file)
@@ -24,8 +24,8 @@ int smpi_coll_tuned_bcast_NTSB(void *buf, int count, MPI_Datatype datatype,
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
   /* source node and destination nodes (same through out the functions) */
   int from = (rank - 1) / 2;
 
   /* source node and destination nodes (same through out the functions) */
   int from = (rank - 1) / 2;
index cb1e007..5304f31 100644 (file)
@@ -26,8 +26,8 @@ int smpi_coll_tuned_bcast_NTSL_Isend(void *buf, int count, MPI_Datatype datatype
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
   /* source node and destination nodes (same through out the functions) */
   int to = (rank + 1) % size;
 
   /* source node and destination nodes (same through out the functions) */
   int to = (rank + 1) % size;
index c9df1a7..146d840 100644 (file)
@@ -26,8 +26,8 @@ int smpi_coll_tuned_bcast_NTSL(void *buf, int count, MPI_Datatype datatype,
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
   /* source node and destination nodes (same through out the functions) */
   int to = (rank + 1) % size;
 
   /* source node and destination nodes (same through out the functions) */
   int to = (rank + 1) % size;
index 5264ca3..b551d84 100644 (file)
@@ -23,14 +23,14 @@ int smpi_coll_tuned_bcast_SMP_binary(void *buf, int count,
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  rank = comm->rank();
+  size = comm->size();
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int host_num_core=1;
   }
   int host_num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    host_num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    host_num_core = comm->get_intra_comm()->size();
   }else{
     //implementation buggy in this case
     return smpi_coll_tuned_bcast_mpich( buf , count, datatype,
   }else{
     //implementation buggy in this case
     return smpi_coll_tuned_bcast_mpich( buf , count, datatype,
index 6053e49..9117219 100644 (file)
@@ -16,15 +16,15 @@ int smpi_coll_tuned_bcast_SMP_binomial(void *buf, int count,
   MPI_Status status;
   int tag = COLL_TAG_BCAST;
 
   MPI_Status status;
   int tag = COLL_TAG_BCAST;
 
-  size = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  size = comm->size();
+  rank = comm->rank();
 
 
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }else{
     //implementation buggy in this case
     return smpi_coll_tuned_bcast_mpich( buf , count, datatype,
   }else{
     //implementation buggy in this case
     return smpi_coll_tuned_bcast_mpich( buf , count, datatype,
index b3f9b6a..049682d 100644 (file)
@@ -22,14 +22,14 @@ int smpi_coll_tuned_bcast_SMP_linear(void *buf, int count,
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  rank = comm->rank();
+  size = comm->size();
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   int num_core=1;
   }
   int num_core=1;
-  if (smpi_comm_is_uniform(comm)){
-    num_core = smpi_comm_size(smpi_comm_get_intra_comm(comm));
+  if (comm->is_uniform()){
+    num_core = comm->get_intra_comm()->size();
   }else{
     //implementation buggy in this case
     return smpi_coll_tuned_bcast_mpich( buf , count, datatype,
   }else{
     //implementation buggy in this case
     return smpi_coll_tuned_bcast_mpich( buf , count, datatype,
index 8e33648..2b3a0a6 100644 (file)
@@ -55,8 +55,8 @@ int smpi_coll_tuned_bcast_arrival_pattern_aware_wait(void *buf, int count,
 
 
 
 
 
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
 
   /* segment is segment size in number of elements (not bytes) */
 
 
   /* segment is segment size in number of elements (not bytes) */
index 09fbbcd..03e4f30 100644 (file)
@@ -45,8 +45,8 @@ int smpi_coll_tuned_bcast_arrival_pattern_aware(void *buf, int count,
 
 
 
 
 
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
 
   /* segment is segment size in number of elements (not bytes) */
 
 
   /* segment is segment size in number of elements (not bytes) */
index ff7b10b..4d0b24a 100644 (file)
@@ -53,8 +53,8 @@ int smpi_coll_tuned_bcast_arrival_scatter(void *buf, int count,
   /* source and destination */
   int to, from;
 
   /* source and destination */
   int to, from;
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
   /* message too small */
   if (count < size) {
 
   /* message too small */
   if (count < size) {
index 955a814..647ce9e 100644 (file)
@@ -76,8 +76,8 @@ smpi_coll_tuned_bcast_binomial_tree(void *buff, int count,
   int src, dst, rank, num_procs, mask, relative_rank;
   int tag = COLL_TAG_BCAST;
 
   int src, dst, rank, num_procs, mask, relative_rank;
   int tag = COLL_TAG_BCAST;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   relative_rank = (rank >= root) ? rank - root : rank - root + num_procs;
 
 
   relative_rank = (rank >= root) ? rank - root : rank - root + num_procs;
 
index 9b94eb2..9246b51 100644 (file)
@@ -27,8 +27,8 @@ smpi_coll_tuned_bcast_flattree_pipeline(void *buff, int count,
     XBT_WARN("MPI_bcast_flattree_pipeline use default MPI_bcast_flattree.");
     return smpi_coll_tuned_bcast_flattree(buff, count, data_type, root, comm);
   }
     XBT_WARN("MPI_bcast_flattree_pipeline use default MPI_bcast_flattree.");
     return smpi_coll_tuned_bcast_flattree(buff, count, data_type, root, comm);
   }
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   MPI_Request *request_array;
   MPI_Status *status_array;
 
   MPI_Request *request_array;
   MPI_Status *status_array;
index 5d4708d..bafa90d 100644 (file)
@@ -16,8 +16,8 @@ smpi_coll_tuned_bcast_flattree(void *buff, int count, MPI_Datatype data_type,
   int i, rank, num_procs;
   int tag = COLL_TAG_BCAST;
 
   int i, rank, num_procs;
   int tag = COLL_TAG_BCAST;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
 
   if (rank != root) {
     smpi_mpi_recv(buff, count, data_type, root, tag, comm, MPI_STATUS_IGNORE);
 
   if (rank != root) {
     smpi_mpi_recv(buff, count, data_type, root, tag, comm, MPI_STATUS_IGNORE);
index 21302b7..4b7f8ba 100644 (file)
@@ -87,8 +87,8 @@ int smpi_coll_tuned_bcast_mvapich2_inter_node(void *buffer,
     int leader_root, leader_of_root;
 
 
     int leader_root, leader_of_root;
 
 
-    rank = smpi_comm_rank(comm);
-    //comm_size = smpi_comm_size(comm);
+    rank = comm->rank();
+    //comm_size = comm->size();
 
 
     if (MV2_Bcast_function==NULL){
 
 
     if (MV2_Bcast_function==NULL){
@@ -99,23 +99,23 @@ int smpi_coll_tuned_bcast_mvapich2_inter_node(void *buffer,
       MV2_Bcast_intra_node_function= smpi_coll_tuned_bcast_mpich;
     }
     
       MV2_Bcast_intra_node_function= smpi_coll_tuned_bcast_mpich;
     }
     
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
     
     }
     
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_rank = smpi_comm_rank(shmem_comm);
-    local_size = smpi_comm_size(shmem_comm);
+    shmem_comm = comm->get_intra_comm();
+    local_rank = shmem_comm->rank();
+    local_size = shmem_comm->size();
 
 
-    leader_comm = smpi_comm_get_leaders_comm(comm);
+    leader_comm = comm->get_leaders_comm();
 
     if ((local_rank == 0) && (local_size > 1)) {
 
     if ((local_rank == 0) && (local_size > 1)) {
-      global_rank = smpi_comm_rank(leader_comm);
+      global_rank = leader_comm->rank();
     }
 
     }
 
-    int* leaders_map = smpi_comm_get_leaders_map(comm);
-    leader_of_root = smpi_comm_group(comm)->rank(leaders_map[root]);
-    leader_root = smpi_comm_group(leader_comm)->rank(leaders_map[root]);
+    int* leaders_map = comm->get_leaders_map();
+    leader_of_root = comm->group()->rank(leaders_map[root]);
+    leader_root = leader_comm->group()->rank(leaders_map[root]);
     
     
     if (local_size > 1) {
     
     
     if (local_size > 1) {
@@ -141,7 +141,7 @@ int smpi_coll_tuned_bcast_mvapich2_inter_node(void *buffer,
 #endif
 /*
     if (local_rank == 0) {
 #endif
 /*
     if (local_rank == 0) {
-        leader_comm = smpi_comm_get_leaders_comm(comm);
+        leader_comm = comm->get_leaders_comm();
         root = leader_root;
     }
 
         root = leader_root;
     }
 
@@ -188,12 +188,12 @@ int smpi_coll_tuned_bcast_mvapich2_knomial_intra_node(void *buffer,
       MV2_Bcast_intra_node_function= smpi_coll_tuned_bcast_mpich;
     }
     
       MV2_Bcast_intra_node_function= smpi_coll_tuned_bcast_mpich;
     }
     
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
     
     }
     
-    local_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    local_size = comm->size();
+    rank = comm->rank();
 
 
     reqarray=(MPI_Request *)xbt_malloc(2 * mv2_intra_node_knomial_factor * sizeof (MPI_Request));
 
 
     reqarray=(MPI_Request *)xbt_malloc(2 * mv2_intra_node_knomial_factor * sizeof (MPI_Request));
@@ -269,12 +269,12 @@ int smpi_coll_tuned_bcast_mvapich2_intra_node(void *buffer,
       MV2_Bcast_intra_node_function= smpi_coll_tuned_bcast_mpich;
     }
     
       MV2_Bcast_intra_node_function= smpi_coll_tuned_bcast_mpich;
     }
     
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
     
     }
     
-    comm_size = smpi_comm_size(comm);
-   // rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+   // rank = comm->rank();
 /*
     if (HANDLE_GET_KIND(datatype) == HANDLE_KIND_BUILTIN)*/
         is_contig = 1;
 /*
     if (HANDLE_GET_KIND(datatype) == HANDLE_KIND_BUILTIN)*/
         is_contig = 1;
@@ -330,7 +330,7 @@ int smpi_coll_tuned_bcast_mvapich2_intra_node(void *buffer,
 /*            }*/
         }
 
 /*            }*/
         }
 
-        shmem_comm = smpi_comm_get_intra_comm(comm);
+        shmem_comm = comm->get_intra_comm();
         if (!is_contig || !is_homogeneous) {
             mpi_errno =
                 MPIR_Bcast_inter_node_helper_MV2(tmp_buf, nbytes, MPI_BYTE,
         if (!is_contig || !is_homogeneous) {
             mpi_errno =
                 MPIR_Bcast_inter_node_helper_MV2(tmp_buf, nbytes, MPI_BYTE,
index 33c4344..a558250 100644 (file)
@@ -42,8 +42,8 @@ int smpi_coll_tuned_bcast_ompi_pipeline( void* buffer,
      */
     type_size = smpi_datatype_size(datatype);
 
      */
     type_size = smpi_datatype_size(datatype);
 
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
     if(size==1)return MPI_SUCCESS;
 
 
     if(size==1)return MPI_SUCCESS;
 
 
@@ -72,7 +72,7 @@ int smpi_coll_tuned_bcast_ompi_pipeline( void* buffer,
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, type_size, count_by_segment );
 
     XBT_DEBUG("coll:tuned:bcast_intra_pipeline rank %d ss %5zu type_size %lu count_by_segment %d",
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, type_size, count_by_segment );
 
     XBT_DEBUG("coll:tuned:bcast_intra_pipeline rank %d ss %5zu type_size %lu count_by_segment %d",
-                 smpi_comm_rank(comm), segsize, (unsigned long)type_size, count_by_segment);
+                 comm->rank(), segsize, (unsigned long)type_size, count_by_segment);
 
 
 
 
 
 
index c49d2db..fdf9b1f 100644 (file)
@@ -84,8 +84,8 @@ smpi_coll_tuned_bcast_ompi_split_bintree ( void* buffer,
 //    mca_coll_tuned_module_t *tuned_module = (mca_coll_tuned_module_t*) module;
 //    mca_coll_tuned_comm_t *data = tuned_module->tuned_data;
 
 //    mca_coll_tuned_module_t *tuned_module = (mca_coll_tuned_module_t*) module;
 //    mca_coll_tuned_comm_t *data = tuned_module->tuned_data;
 
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
 
     //compute again segsize
 
 
     //compute again segsize
index 87d84ca..d8091d9 100644 (file)
@@ -79,8 +79,8 @@ smpi_coll_tuned_bcast_scatter_LR_allgather(void *buff, int count,
   int scatter_size, left, right, next_src, *recv_counts, *disps;
   int tag = COLL_TAG_BCAST;
 
   int scatter_size, left, right, next_src, *recv_counts, *disps;
   int tag = COLL_TAG_BCAST;
 
-  rank = smpi_comm_rank(comm);
-  num_procs = smpi_comm_size(comm);
+  rank = comm->rank();
+  num_procs = comm->size();
   extent = smpi_datatype_get_extent(data_type);
 
 
   extent = smpi_datatype_get_extent(data_type);
 
 
index d4f8cab..fa49229 100644 (file)
@@ -13,8 +13,8 @@ static int scatter_for_bcast(
     int mpi_errno = MPI_SUCCESS;
     int scatter_size, curr_size, recv_size = 0, send_size;
 
     int mpi_errno = MPI_SUCCESS;
     int scatter_size, curr_size, recv_size = 0, send_size;
 
-    comm_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    rank = comm->rank();
     relative_rank = (rank >= root) ? rank - root : rank - root + comm_size;
 
     /* use long message algorithm: binomial tree scatter followed by an allgather */
     relative_rank = (rank >= root) ? rank - root : rank - root + comm_size;
 
     /* use long message algorithm: binomial tree scatter followed by an allgather */
@@ -114,8 +114,8 @@ smpi_coll_tuned_bcast_scatter_rdb_allgather (
     MPI_Aint true_extent, true_lb;
     void *tmp_buf;
 
     MPI_Aint true_extent, true_lb;
     void *tmp_buf;
 
-    comm_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    rank = comm->rank();
     relative_rank = (rank >= root) ? rank - root : rank - root + comm_size;
 
     /* If there is only one process, return */
     relative_rank = (rank >= root) ? rank - root : rank - root + comm_size;
 
     /* If there is only one process, return */
index a5c53b1..b04da38 100644 (file)
@@ -95,8 +95,8 @@ ompi_coll_tuned_topo_build_tree( int fanout,
     /* 
      * Get size and rank of the process in this communicator 
      */
     /* 
      * Get size and rank of the process in this communicator 
      */
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     tree = (ompi_coll_tree_t*)malloc(sizeof(ompi_coll_tree_t));
     if (!tree) {
 
     tree = (ompi_coll_tree_t*)malloc(sizeof(ompi_coll_tree_t));
     if (!tree) {
@@ -195,8 +195,8 @@ ompi_coll_tuned_topo_build_in_order_bintree( MPI_Comm comm )
     /* 
      * Get size and rank of the process in this communicator 
      */
     /* 
      * Get size and rank of the process in this communicator 
      */
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     tree = (ompi_coll_tree_t*)malloc(sizeof(ompi_coll_tree_t));
     if (!tree) {
 
     tree = (ompi_coll_tree_t*)malloc(sizeof(ompi_coll_tree_t));
     if (!tree) {
@@ -338,8 +338,8 @@ ompi_coll_tuned_topo_build_bmtree( MPI_Comm comm,
     /* 
      * Get size and rank of the process in this communicator 
      */
     /* 
      * Get size and rank of the process in this communicator 
      */
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     index = rank -root;
 
 
     index = rank -root;
 
@@ -419,8 +419,8 @@ ompi_coll_tuned_topo_build_in_order_bmtree( MPI_Comm comm,
     /* 
      * Get size and rank of the process in this communicator 
      */
     /* 
      * Get size and rank of the process in this communicator 
      */
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     vrank = (rank - root + size) % size;
 
 
     vrank = (rank - root + size) % size;
 
@@ -481,8 +481,8 @@ ompi_coll_tuned_topo_build_chain( int fanout,
     /* 
      * Get size and rank of the process in this communicator 
      */
     /* 
      * Get size and rank of the process in this communicator 
      */
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     if( fanout < 1 ) {
         XBT_DEBUG("coll:tuned:topo:build_chain WARNING invalid fanout of ZERO, forcing to 1 (pipeline)!");
 
     if( fanout < 1 ) {
         XBT_DEBUG("coll:tuned:topo:build_chain WARNING invalid fanout of ZERO, forcing to 1 (pipeline)!");
index 2202b9e..9ca1192 100644 (file)
@@ -149,11 +149,11 @@ int smpi_coll_tuned_gather_mvapich2_two_level(void *sendbuf,
     if(MV2_Gather_intra_node_function==NULL)
       MV2_Gather_intra_node_function=smpi_coll_tuned_gather_mpich;
     
     if(MV2_Gather_intra_node_function==NULL)
       MV2_Gather_intra_node_function=smpi_coll_tuned_gather_mpich;
     
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
     }
-    comm_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    rank = comm->rank();
 
     if (((rank == root) && (recvcnt == 0)) ||
         ((rank != root) && (sendcnt == 0))) {
 
     if (((rank == root) && (recvcnt == 0)) ||
         ((rank != root) && (sendcnt == 0))) {
@@ -175,19 +175,19 @@ int smpi_coll_tuned_gather_mvapich2_two_level(void *sendbuf,
 
     /* extract the rank,size information for the intra-node
      * communicator */
 
     /* extract the rank,size information for the intra-node
      * communicator */
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_rank = smpi_comm_rank(shmem_comm);
-    local_size = smpi_comm_size(shmem_comm);
+    shmem_comm = comm->get_intra_comm();
+    local_rank = shmem_comm->rank();
+    local_size = shmem_comm->size();
     
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader
          * communicator */
     
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader
          * communicator */
-        leader_comm = smpi_comm_get_leaders_comm(comm);
+        leader_comm = comm->get_leaders_comm();
         if(leader_comm==MPI_COMM_NULL){
           leader_comm = MPI_COMM_WORLD;
         }
         if(leader_comm==MPI_COMM_NULL){
           leader_comm = MPI_COMM_WORLD;
         }
-        leader_comm_size = smpi_comm_size(leader_comm);
-        leader_comm_rank = smpi_comm_rank(leader_comm);
+        leader_comm_size = leader_comm->size();
+        leader_comm_rank = leader_comm->size();
     }
 
     if (rank == root) {
     }
 
     if (rank == root) {
@@ -263,14 +263,14 @@ int smpi_coll_tuned_gather_mvapich2_two_level(void *sendbuf,
                                                  );
         }
     }
                                                  );
         }
     }
-    leader_comm = smpi_comm_get_leaders_comm(comm);
-    int* leaders_map = smpi_comm_get_leaders_map(comm);
-    leader_of_root = smpi_comm_group(comm)->rank(leaders_map[root]);
-    leader_root = smpi_comm_group(leader_comm)->rank(leaders_map[root]);
+    leader_comm = comm->get_leaders_comm();
+    int* leaders_map = comm->get_leaders_map();
+    leader_of_root = comm->group()->rank(leaders_map[root]);
+    leader_root = leader_comm->group()->rank(leaders_map[root]);
     /* leader_root is the rank of the leader of the root in leader_comm. 
      * leader_root is to be used as the root of the inter-leader gather ops 
      */
     /* leader_root is the rank of the leader of the root in leader_comm. 
      * leader_root is to be used as the root of the inter-leader gather ops 
      */
-    if (!smpi_comm_is_uniform(comm)) {
+    if (!comm->is_uniform()) {
         if (local_rank == 0) {
             int *displs = NULL;
             int *recvcnts = NULL;
         if (local_rank == 0) {
             int *displs = NULL;
             int *recvcnts = NULL;
@@ -302,7 +302,7 @@ int smpi_coll_tuned_gather_mvapich2_two_level(void *sendbuf,
                 }
             }
 
                 }
             }
 
-            node_sizes = smpi_comm_get_non_uniform_map(comm);
+            node_sizes = comm->get_non_uniform_map();
 
             if (leader_comm_rank == leader_root) {
                 displs =  static_cast<int *>(xbt_malloc(sizeof (int) * leader_comm_size));
 
             if (leader_comm_rank == leader_root) {
                 displs =  static_cast<int *>(xbt_malloc(sizeof (int) * leader_comm_size));
index 46dc36e..e7d1cff 100644 (file)
@@ -47,8 +47,8 @@ smpi_coll_tuned_gather_ompi_binomial(void *sbuf, int scount,
     MPI_Aint rextent, rlb, rtrue_lb, rtrue_extent;
 
 
     MPI_Aint rextent, rlb, rtrue_lb, rtrue_extent;
 
 
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     XBT_DEBUG(
                 "smpi_coll_tuned_gather_ompi_binomial rank %d", rank);
 
     XBT_DEBUG(
                 "smpi_coll_tuned_gather_ompi_binomial rank %d", rank);
@@ -211,8 +211,8 @@ smpi_coll_tuned_gather_ompi_linear_sync(void *sbuf, int scount,
     MPI_Aint lb;
 
     int first_segment_size=0;
     MPI_Aint lb;
 
     int first_segment_size=0;
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
     
     size_t dsize, block_size;
     if (rank == root) {
     
     size_t dsize, block_size;
     if (rank == root) {
@@ -370,8 +370,8 @@ smpi_coll_tuned_gather_ompi_basic_linear(void *sbuf, int scount,
     MPI_Aint extent;
     MPI_Aint lb;
 
     MPI_Aint extent;
     MPI_Aint lb;
 
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     /* Everyone but root sends data and returns. */
     XBT_DEBUG(
 
     /* Everyone but root sends data and returns. */
     XBT_DEBUG(
index 60aa06c..c2560e5 100644 (file)
@@ -27,8 +27,8 @@ int smpi_coll_tuned_reduce_NTSL(void *buf, void *rbuf, int count,
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
   MPI_Aint extent;
   extent = smpi_datatype_get_extent(datatype);
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
   /* source node and destination nodes (same through out the functions) */
   int to = (rank - 1 + size) % size;
 
   /* source node and destination nodes (same through out the functions) */
   int to = (rank - 1 + size) % size;
index e528e9a..7aaf904 100644 (file)
@@ -24,7 +24,7 @@ int smpi_coll_tuned_reduce_arrival_pattern_aware(void *buf, void *rbuf,
                                                  MPI_Op op, int root,
                                                  MPI_Comm comm)
 {
                                                  MPI_Op op, int root,
                                                  MPI_Comm comm)
 {
-  int rank = smpi_comm_rank(comm);
+  int rank = comm->rank();
   int tag = -COLL_TAG_REDUCE;
   MPI_Status status;
   MPI_Request request;
   int tag = -COLL_TAG_REDUCE;
   MPI_Status status;
   MPI_Request request;
@@ -35,7 +35,7 @@ int smpi_coll_tuned_reduce_arrival_pattern_aware(void *buf, void *rbuf,
 
   MPI_Status temp_status_array[MAX_NODE];
 
 
   MPI_Status temp_status_array[MAX_NODE];
 
-  int size = smpi_comm_size(comm);
+  int size = comm->size();
   int i;
 
   int sent_count;
   int i;
 
   int sent_count;
index 180a07d..d9255a8 100644 (file)
@@ -22,8 +22,8 @@ int smpi_coll_tuned_reduce_binomial(void *sendbuf, void *recvbuf, int count,
   MPI_Aint true_lb, true_extent;
   if (count == 0)
     return 0;
   MPI_Aint true_lb, true_extent;
   if (count == 0)
     return 0;
-  rank = smpi_comm_rank(comm);
-  comm_size = smpi_comm_size(comm);
+  rank = comm->rank();
+  comm_size = comm->size();
 
   extent = smpi_datatype_get_extent(datatype);
 
 
   extent = smpi_datatype_get_extent(datatype);
 
index 685fc2f..a12009c 100644 (file)
@@ -20,8 +20,8 @@ smpi_coll_tuned_reduce_flat_tree(void *sbuf, void *rbuf, int count,
   char *inbuf;
   MPI_Status status;
 
   char *inbuf;
   MPI_Status status;
 
-  rank = smpi_comm_rank(comm);
-  size = smpi_comm_size(comm);
+  rank = comm->rank();
+  size = comm->size();
 
   /* If not root, send data to the root. */
   extent = smpi_datatype_get_extent(dtype);
 
   /* If not root, send data to the root. */
   extent = smpi_datatype_get_extent(dtype);
index 45418ee..39f69bb 100644 (file)
@@ -56,8 +56,8 @@ static int MPIR_Reduce_knomial_trace(int root, int reduce_knomial_factor,
     int orig_mask=0x1; 
     int recv_iter=0, send_iter=0;
     int *knomial_reduce_src_array=NULL;
     int orig_mask=0x1; 
     int recv_iter=0, send_iter=0;
     int *knomial_reduce_src_array=NULL;
-    comm_size =  smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size =  comm->size();
+    rank = comm->rank();
 
     lroot = root;
     relative_rank = (rank - lroot + comm_size) % comm_size;
 
     lroot = root;
     relative_rank = (rank - lroot + comm_size) % comm_size;
@@ -138,7 +138,7 @@ int smpi_coll_tuned_reduce_mvapich2_knomial (
 
     if (count == 0) return MPI_SUCCESS;
 
 
     if (count == 0) return MPI_SUCCESS;
 
-    rank = smpi_comm_rank(comm);
+    rank = comm->rank();
 
     /* Create a temporary buffer */
 
 
     /* Create a temporary buffer */
 
index d810f75..70e7a8f 100644 (file)
@@ -97,20 +97,20 @@ int smpi_coll_tuned_reduce_mvapich2_two_level( void *sendbuf,
     if(MV2_Reduce_intra_function==NULL)
       MV2_Reduce_intra_function=smpi_coll_tuned_reduce_mpich;
 
     if(MV2_Reduce_intra_function==NULL)
       MV2_Reduce_intra_function=smpi_coll_tuned_reduce_mpich;
 
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
   
     }
   
-    my_rank = smpi_comm_rank(comm);
-    total_size = smpi_comm_size(comm);
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_rank = smpi_comm_rank(shmem_comm);
-    local_size = smpi_comm_size(shmem_comm);
+    my_rank = comm->rank();
+    total_size = comm->size();
+    shmem_comm = comm->get_intra_comm();
+    local_rank = shmem_comm->rank();
+    local_size = shmem_comm->size();
     
     
-    leader_comm = smpi_comm_get_leaders_comm(comm);
-    int* leaders_map = smpi_comm_get_leaders_map(comm);
-    leader_of_root = smpi_comm_group(comm)->rank(leaders_map[root]);
-    leader_root = smpi_comm_group(leader_comm)->rank(leaders_map[root]);
+    leader_comm = comm->get_leaders_comm();
+    int* leaders_map = comm->get_leaders_map();
+    leader_of_root = comm->group()->rank(leaders_map[root]);
+    leader_root = leader_comm->group()->rank(leaders_map[root]);
 
     is_commutative=smpi_op_is_commute(op);
 
 
     is_commutative=smpi_op_is_commute(op);
 
@@ -188,12 +188,12 @@ int smpi_coll_tuned_reduce_mvapich2_two_level( void *sendbuf,
     
 
     if (local_rank == 0) {
     
 
     if (local_rank == 0) {
-        leader_comm = smpi_comm_get_leaders_comm(comm);
+        leader_comm = comm->get_leaders_comm();
         if(leader_comm==MPI_COMM_NULL){
           leader_comm = MPI_COMM_WORLD;
         }
         if(leader_comm==MPI_COMM_NULL){
           leader_comm = MPI_COMM_WORLD;
         }
-        leader_comm_size = smpi_comm_size(leader_comm);
-        leader_comm_rank = smpi_comm_rank(leader_comm);
+        leader_comm_size = leader_comm->size();
+        leader_comm_rank = leader_comm->rank();
         tmp_buf=(void *)smpi_get_tmp_sendbuffer(count *
                             (MAX(extent, true_extent)));
         tmp_buf = (void *) ((char *) tmp_buf - true_lb);
         tmp_buf=(void *)smpi_get_tmp_sendbuffer(count *
                             (MAX(extent, true_extent)));
         tmp_buf = (void *) ((char *) tmp_buf - true_lb);
index 3c51282..faf37ef 100644 (file)
@@ -69,7 +69,7 @@ int smpi_coll_tuned_ompi_reduce_generic( void* sendbuf, void* recvbuf, int origi
 
     XBT_DEBUG( "coll:tuned:reduce_generic count %d, msg size %ld, segsize %ld, max_requests %d", original_count, (unsigned long)(num_segments * segment_increment), (unsigned long)segment_increment, max_outstanding_reqs);
 
 
     XBT_DEBUG( "coll:tuned:reduce_generic count %d, msg size %ld, segsize %ld, max_requests %d", original_count, (unsigned long)(num_segments * segment_increment), (unsigned long)segment_increment, max_outstanding_reqs);
 
-    rank = smpi_comm_rank(comm);
+    rank = comm->rank();
 
     /* non-leaf nodes - wait for children to send me data & forward up 
        (if needed) */
 
     /* non-leaf nodes - wait for children to send me data & forward up 
        (if needed) */
@@ -333,9 +333,9 @@ int smpi_coll_tuned_reduce_ompi_chain( void *sendbuf, void *recvbuf, int count,
     uint32_t segsize=64*1024;
     int segcount = count;
     size_t typelng;
     uint32_t segsize=64*1024;
     int segcount = count;
     size_t typelng;
-    int fanout = smpi_comm_size(comm)/2;
+    int fanout = comm->size()/2;
 
 
-    XBT_DEBUG("coll:tuned:reduce_intra_chain rank %d fo %d ss %5d", smpi_comm_rank(comm), fanout, segsize);
+    XBT_DEBUG("coll:tuned:reduce_intra_chain rank %d fo %d ss %5d", comm->rank(), fanout, segsize);
 
     /**
      * Determine number of segments and number of elements
 
     /**
      * Determine number of segments and number of elements
@@ -372,7 +372,7 @@ int smpi_coll_tuned_reduce_ompi_pipeline( void *sendbuf, void *recvbuf,
     const double a4 =  0.0033 / 1024.0; /* [1/B] */
     const double b4 =  1.6761;
     typelng= smpi_datatype_size( datatype);
     const double a4 =  0.0033 / 1024.0; /* [1/B] */
     const double b4 =  1.6761;
     typelng= smpi_datatype_size( datatype);
-    int communicator_size = smpi_comm_size(comm);
+    int communicator_size = comm->size();
     size_t message_size = typelng * count; 
 
     if (communicator_size > (a2 * message_size + b2)) {
     size_t message_size = typelng * count; 
 
     if (communicator_size > (a2 * message_size + b2)) {
@@ -387,7 +387,7 @@ int smpi_coll_tuned_reduce_ompi_pipeline( void *sendbuf, void *recvbuf,
     }
 
     XBT_DEBUG("coll:tuned:reduce_intra_pipeline rank %d ss %5d",
     }
 
     XBT_DEBUG("coll:tuned:reduce_intra_pipeline rank %d ss %5d",
-                 smpi_comm_rank(comm), segsize);
+                 comm->rank(), segsize);
 
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, typelng, segcount );
 
 
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, typelng, segcount );
 
@@ -418,7 +418,7 @@ int smpi_coll_tuned_reduce_ompi_binary( void *sendbuf, void *recvbuf,
     segsize = 32*1024;
 
     XBT_DEBUG("coll:tuned:reduce_intra_binary rank %d ss %5d",
     segsize = 32*1024;
 
     XBT_DEBUG("coll:tuned:reduce_intra_binary rank %d ss %5d",
-                 smpi_comm_rank(comm), segsize);
+                 comm->rank(), segsize);
 
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, typelng, segcount );
 
 
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, typelng, segcount );
 
@@ -448,7 +448,7 @@ int smpi_coll_tuned_reduce_ompi_binomial( void *sendbuf, void *recvbuf,
      * sent per operation
      */
     typelng= smpi_datatype_size( datatype);
      * sent per operation
      */
     typelng= smpi_datatype_size( datatype);
-    int communicator_size = smpi_comm_size(comm);
+    int communicator_size = comm->size();
     size_t message_size = typelng * count; 
     if (((communicator_size < 8) && (message_size < 20480)) ||
                (message_size < 2048) || (count <= 1)) {
     size_t message_size = typelng * count; 
     if (((communicator_size < 8) && (message_size < 20480)) ||
                (message_size < 2048) || (count <= 1)) {
@@ -460,7 +460,7 @@ int smpi_coll_tuned_reduce_ompi_binomial( void *sendbuf, void *recvbuf,
     }
 
     XBT_DEBUG("coll:tuned:reduce_intra_binomial rank %d ss %5d",
     }
 
     XBT_DEBUG("coll:tuned:reduce_intra_binomial rank %d ss %5d",
-                 smpi_comm_rank(comm), segsize);
+                 comm->rank(), segsize);
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, typelng, segcount );
 
     return smpi_coll_tuned_ompi_reduce_generic( sendbuf, recvbuf, count, datatype, 
     COLL_TUNED_COMPUTED_SEGCOUNT( segsize, typelng, segcount );
 
     return smpi_coll_tuned_ompi_reduce_generic( sendbuf, recvbuf, count, datatype, 
@@ -489,8 +489,8 @@ int smpi_coll_tuned_reduce_ompi_in_order_binary( void *sendbuf, void *recvbuf,
     void *use_this_sendbuf = NULL, *use_this_recvbuf = NULL;
     size_t typelng;
 
     void *use_this_sendbuf = NULL, *use_this_recvbuf = NULL;
     size_t typelng;
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
     XBT_DEBUG("coll:tuned:reduce_intra_in_order_binary rank %d ss %5d",
                  rank, segsize);
 
     XBT_DEBUG("coll:tuned:reduce_intra_in_order_binary rank %d ss %5d",
                  rank, segsize);
 
@@ -604,8 +604,8 @@ smpi_coll_tuned_reduce_ompi_basic_linear(void *sbuf, void *rbuf, int count,
 
     /* Initialize */
 
 
     /* Initialize */
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
 
     XBT_DEBUG("coll:tuned:reduce_intra_basic_linear rank %d", rank);
 
 
     XBT_DEBUG("coll:tuned:reduce_intra_basic_linear rank %d", rank);
 
index 38db760..6256be7 100644 (file)
@@ -31,8 +31,8 @@ int smpi_coll_tuned_reduce_scatter_gather(void *sendbuf, void *recvbuf,
 
   if (count == 0)
     return 0;
 
   if (count == 0)
     return 0;
-  rank = smpi_comm_rank(comm);
-  comm_size = smpi_comm_size(comm);
+  rank = comm->rank();
+  comm_size = comm->size();
   
 
 
   
 
 
index b01fbd0..2692b50 100644 (file)
@@ -32,8 +32,8 @@ int smpi_coll_tuned_reduce_scatter_mpich_pair(void *sendbuf, void *recvbuf, int
     int mpi_errno = MPI_SUCCESS;
     int total_count, dst, src;
     int is_commutative;
     int mpi_errno = MPI_SUCCESS;
     int total_count, dst, src;
     int is_commutative;
-    comm_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    rank = comm->rank();
 
     extent =smpi_datatype_get_extent(datatype);
     smpi_datatype_extent(datatype, &true_lb, &true_extent);
 
     extent =smpi_datatype_get_extent(datatype);
     smpi_datatype_extent(datatype, &true_lb, &true_extent);
@@ -152,8 +152,8 @@ int smpi_coll_tuned_reduce_scatter_mpich_noncomm(void *sendbuf, void *recvbuf, i
                               MPI_Datatype datatype, MPI_Op op, MPI_Comm comm)
 {
     int mpi_errno = MPI_SUCCESS;
                               MPI_Datatype datatype, MPI_Op op, MPI_Comm comm)
 {
     int mpi_errno = MPI_SUCCESS;
-    int comm_size = smpi_comm_size(comm) ;
-    int rank = smpi_comm_rank(comm);
+    int comm_size = comm->size() ;
+    int rank = comm->rank();
     int pof2;
     int log2_comm_size;
     int i, k;
     int pof2;
     int log2_comm_size;
     int i, k;
@@ -279,8 +279,8 @@ int smpi_coll_tuned_reduce_scatter_mpich_rdb(void *sendbuf, void *recvbuf, int r
     int received;
     MPI_Datatype sendtype, recvtype;
     int nprocs_completed, tmp_mask, tree_root, is_commutative=0;
     int received;
     MPI_Datatype sendtype, recvtype;
     int nprocs_completed, tmp_mask, tree_root, is_commutative=0;
-    comm_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    rank = comm->rank();
 
     extent =smpi_datatype_get_extent(datatype);
     smpi_datatype_extent(datatype, &true_lb, &true_extent);
 
     extent =smpi_datatype_get_extent(datatype);
     smpi_datatype_extent(datatype, &true_lb, &true_extent);
index e303d20..8186229 100644 (file)
@@ -58,8 +58,8 @@ smpi_coll_tuned_reduce_scatter_ompi_basic_recursivehalving(void *sbuf,
     char *result_buf = NULL, *result_buf_free = NULL;
    
     /* Initialize */
     char *result_buf = NULL, *result_buf_free = NULL;
    
     /* Initialize */
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
    
     XBT_DEBUG("coll:tuned:reduce_scatter_ompi_basic_recursivehalving, rank %d", rank);
     if(!smpi_op_is_commute(op))
    
     XBT_DEBUG("coll:tuned:reduce_scatter_ompi_basic_recursivehalving, rank %d", rank);
     if(!smpi_op_is_commute(op))
@@ -374,8 +374,8 @@ smpi_coll_tuned_reduce_scatter_ompi_ring(void *sbuf, void *rbuf, int *rcounts,
     ptrdiff_t true_lb, true_extent, lb, extent, max_real_segsize;
     MPI_Request reqs[2] = {NULL, NULL};
 
     ptrdiff_t true_lb, true_extent, lb, extent, max_real_segsize;
     MPI_Request reqs[2] = {NULL, NULL};
 
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     XBT_DEBUG(  "coll:tuned:reduce_scatter_ompi_ring rank %d, size %d", 
                  rank, size);
 
     XBT_DEBUG(  "coll:tuned:reduce_scatter_ompi_ring rank %d, size %d", 
                  rank, size);
index 09f5615..e459ad0 100644 (file)
@@ -66,11 +66,11 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_direct(void *sendbuf,
     if(MV2_Scatter_intra_function==NULL)
       MV2_Scatter_intra_function=smpi_coll_tuned_scatter_mpich;
     
     if(MV2_Scatter_intra_function==NULL)
       MV2_Scatter_intra_function=smpi_coll_tuned_scatter_mpich;
     
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
     }
-    comm_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    rank = comm->rank();
 
     if (((rank == root) && (recvcnt == 0))
         || ((rank != root) && (sendcnt == 0))) {
 
     if (((rank == root) && (recvcnt == 0))
         || ((rank != root) && (sendcnt == 0))) {
@@ -79,16 +79,16 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_direct(void *sendbuf,
 
     /* extract the rank,size information for the intra-node
      * communicator */
 
     /* extract the rank,size information for the intra-node
      * communicator */
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_rank = smpi_comm_rank(shmem_comm);
-    local_size = smpi_comm_size(shmem_comm);
+    shmem_comm = comm->get_intra_comm();
+    local_rank = shmem_comm->rank();
+    local_size = shmem_comm->size();
 
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader
          * communicator */
 
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader
          * communicator */
-        leader_comm = smpi_comm_get_leaders_comm(comm);
-        leader_comm_size = smpi_comm_size(leader_comm);
-        leader_comm_rank = smpi_comm_rank(leader_comm);
+        leader_comm = comm->get_leaders_comm();
+        leader_comm_size = leader_comm->size();
+        leader_comm_rank = leader_comm->rank();
     }
 
     if (local_size == comm_size) {
     }
 
     if (local_size == comm_size) {
@@ -112,10 +112,10 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_direct(void *sendbuf,
             tmp_buf = smpi_get_tmp_sendbuffer(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 = smpi_comm_group(comm)->rank(leaders_map[root]);
-        leader_root = smpi_comm_group(leader_comm)->rank(leaders_map[root]);
+        leader_comm = comm->get_leaders_comm();
+        int* leaders_map = comm->get_leaders_map();
+        leader_of_root = comm->group()->rank(leaders_map[root]);
+        leader_root = leader_comm->group()->rank(leaders_map[root]);
         /* leader_root is the rank of the leader of the root in leader_comm.
          * leader_root is to be used as the root of the inter-leader gather ops
          */
         /* leader_root is the rank of the leader of the root in leader_comm.
          * leader_root is to be used as the root of the inter-leader gather ops
          */
@@ -139,12 +139,12 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_direct(void *sendbuf,
         }
 
         if (leader_comm_size > 1 && local_rank == 0) {
         }
 
         if (leader_comm_size > 1 && local_rank == 0) {
-            if (!smpi_comm_is_uniform(comm)) {
+            if (!comm->is_uniform()) {
                 int *displs = NULL;
                 int *sendcnts = NULL;
                 int *node_sizes;
                 int i = 0;
                 int *displs = NULL;
                 int *sendcnts = NULL;
                 int *node_sizes;
                 int i = 0;
-                node_sizes = smpi_comm_get_non_uniform_map(comm);
+                node_sizes = comm->get_non_uniform_map();
 
                 if (root != leader_of_root) {
                     if (leader_comm_rank == leader_root) {
 
                 if (root != leader_of_root) {
                     if (leader_comm_rank == leader_root) {
@@ -250,11 +250,11 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_binomial(void *sendbuf,
     if(MV2_Scatter_intra_function==NULL)
       MV2_Scatter_intra_function=smpi_coll_tuned_scatter_mpich;
     
     if(MV2_Scatter_intra_function==NULL)
       MV2_Scatter_intra_function=smpi_coll_tuned_scatter_mpich;
     
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
     }
-    comm_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    rank = comm->rank();
 
     if (((rank == root) && (recvcnt == 0))
         || ((rank != root) && (sendcnt == 0))) {
 
     if (((rank == root) && (recvcnt == 0))
         || ((rank != root) && (sendcnt == 0))) {
@@ -263,16 +263,16 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_binomial(void *sendbuf,
 
     /* extract the rank,size information for the intra-node
      * communicator */
 
     /* extract the rank,size information for the intra-node
      * communicator */
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_rank = smpi_comm_rank(shmem_comm);
-    local_size = smpi_comm_size(shmem_comm);
+    shmem_comm = comm->get_intra_comm();
+    local_rank = shmem_comm->rank();
+    local_size = shmem_comm->size();
 
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader
          * communicator */
 
     if (local_rank == 0) {
         /* Node leader. Extract the rank, size information for the leader
          * communicator */
-        leader_comm = smpi_comm_get_leaders_comm(comm);
-        leader_comm_size = smpi_comm_size(leader_comm);
-        leader_comm_rank = smpi_comm_rank(leader_comm);
+        leader_comm = comm->get_leaders_comm();
+        leader_comm_size = leader_comm->size();
+        leader_comm_rank = leader_comm->rank();
     }
 
     if (local_size == comm_size) {
     }
 
     if (local_size == comm_size) {
@@ -295,10 +295,10 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_binomial(void *sendbuf,
             /* Node leader, allocate tmp_buffer */
             tmp_buf = smpi_get_tmp_sendbuffer(nbytes * local_size);
         }
             /* Node leader, allocate tmp_buffer */
             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 = smpi_comm_group(comm)->rank(leaders_map[root]);
-        leader_root = smpi_comm_group(leader_comm)->rank(leaders_map[root]);
+        leader_comm = comm->get_leaders_comm();
+        int* leaders_map = comm->get_leaders_map();
+        leader_of_root = comm->group()->rank(leaders_map[root]);
+        leader_root = leader_comm->group()->rank(leaders_map[root]);
         /* leader_root is the rank of the leader of the root in leader_comm.
          * leader_root is to be used as the root of the inter-leader gather ops
          */
         /* leader_root is the rank of the leader of the root in leader_comm.
          * leader_root is to be used as the root of the inter-leader gather ops
          */
@@ -320,12 +320,12 @@ int smpi_coll_tuned_scatter_mvapich2_two_level_binomial(void *sendbuf,
         }
 
         if (leader_comm_size > 1 && local_rank == 0) {
         }
 
         if (leader_comm_size > 1 && local_rank == 0) {
-            if (!smpi_comm_is_uniform(comm)) {
+            if (!comm->is_uniform()) {
                 int *displs = NULL;
                 int *sendcnts = NULL;
                 int *node_sizes;
                 int i = 0;
                 int *displs = NULL;
                 int *sendcnts = NULL;
                 int *node_sizes;
                 int i = 0;
-                node_sizes = smpi_comm_get_non_uniform_map(comm);
+                node_sizes = comm->get_non_uniform_map();
 
                 if (root != leader_of_root) {
                     if (leader_comm_rank == leader_root) {
 
                 if (root != leader_of_root) {
                     if (leader_comm_rank == leader_root) {
index 9eace67..6fb5e0c 100644 (file)
@@ -47,8 +47,8 @@ smpi_coll_tuned_scatter_ompi_binomial(void *sbuf, int scount,
     MPI_Aint sextent, slb, strue_lb, strue_extent; 
     MPI_Aint rextent, rlb, rtrue_lb, rtrue_extent;
 
     MPI_Aint sextent, slb, strue_lb, strue_extent; 
     MPI_Aint rextent, rlb, rtrue_lb, rtrue_extent;
 
-    size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    size = comm->size();
+    rank = comm->rank();
 
     XBT_DEBUG(
                  "smpi_coll_tuned_scatter_ompi_binomial rank %d", rank);
 
     XBT_DEBUG(
                  "smpi_coll_tuned_scatter_ompi_binomial rank %d", rank);
@@ -205,8 +205,8 @@ smpi_coll_tuned_scatter_ompi_basic_linear(void *sbuf, int scount,
 
     /* Initialize */
 
 
     /* Initialize */
 
-    rank = smpi_comm_rank(comm);
-    size = smpi_comm_size(comm);
+    rank = comm->rank();
+    size = comm->size();
 
     /* If not root, receive data. */
 
 
     /* If not root, receive data. */
 
index c79cea5..f6e5240 100644 (file)
           min_coll=i;\
           time_min=time2-time1;\
       }\
           min_coll=i;\
           time_min=time2-time1;\
       }\
-      if(smpi_comm_rank(comm)==0){\
+      if(comm->rank()==0){\
           if(buf_in<max_min){\
               max_min=buf_in;\
               global_coll=i;\
           }\
       }\
   }\
           if(buf_in<max_min){\
               max_min=buf_in;\
               global_coll=i;\
           }\
       }\
   }\
-  if(smpi_comm_rank(comm)==0){\
+  if(comm->rank()==0){\
       XBT_WARN("For rank 0, the quickest was %s : %f , but global was %s : %f at max",mpi_coll_##cat##_description[min_coll].name, time_min,mpi_coll_##cat##_description[global_coll].name, max_min);\
   }else\
       XBT_WARN("For rank 0, the quickest was %s : %f , but global was %s : %f at max",mpi_coll_##cat##_description[min_coll].name, time_min,mpi_coll_##cat##_description[global_coll].name, max_min);\
   }else\
-  XBT_WARN("The quickest %s was %s on rank %d and took %f",#cat,mpi_coll_##cat##_description[min_coll].name, smpi_comm_rank(comm), time_min);\
+  XBT_WARN("The quickest %s was %s on rank %d and took %f",#cat,mpi_coll_##cat##_description[min_coll].name, comm->rank(), time_min);\
   return (min_coll!=-1)?MPI_SUCCESS:MPI_ERR_INTERN;\
 }\
 
   return (min_coll!=-1)?MPI_SUCCESS:MPI_ERR_INTERN;\
 }\
 
index c848923..43138e3 100644 (file)
@@ -2246,13 +2246,13 @@ intel_tuning_table_element intel_alltoallv_table[] =
     size_t block_dsize = total_message_size*smpi_datatype_size(recv_type);
     
 #define SIZECOMP_gather\
     size_t block_dsize = total_message_size*smpi_datatype_size(recv_type);
     
 #define SIZECOMP_gather\
-  int rank = smpi_comm_rank(comm);\
+  int rank = comm->rank();\
   size_t block_dsize = (send_buff == MPI_IN_PLACE || rank ==root) ?\
                 recv_count * smpi_datatype_size(recv_type) :\
                 send_count * smpi_datatype_size(send_type);
 
 #define SIZECOMP_scatter\
   size_t block_dsize = (send_buff == MPI_IN_PLACE || rank ==root) ?\
                 recv_count * smpi_datatype_size(recv_type) :\
                 send_count * smpi_datatype_size(send_type);
 
 #define SIZECOMP_scatter\
-  int rank = smpi_comm_rank(comm);\
+  int rank = comm->rank();\
   size_t block_dsize = (sendbuf == MPI_IN_PLACE || rank !=root ) ?\
                 recvcount * smpi_datatype_size(recvtype) :\
                 sendcount * smpi_datatype_size(sendtype);
   size_t block_dsize = (sendbuf == MPI_IN_PLACE || rank !=root ) ?\
                 recvcount * smpi_datatype_size(recvtype) :\
                 sendcount * smpi_datatype_size(sendtype);
@@ -2263,17 +2263,17 @@ intel_tuning_table_element intel_alltoallv_table[] =
 #define IMPI_COLL_SELECT(cat, ret, args, args2)\
 ret smpi_coll_tuned_ ## cat ## _impi (COLL_UNPAREN args)\
 {\
 #define IMPI_COLL_SELECT(cat, ret, args, args2)\
 ret smpi_coll_tuned_ ## cat ## _impi (COLL_UNPAREN args)\
 {\
-    int comm_size = smpi_comm_size(comm);\
+    int comm_size = comm->size();\
     int i =0;\
     SIZECOMP_ ## cat\
     i=0;\
     int j =0, k=0;\
     int i =0;\
     SIZECOMP_ ## cat\
     i=0;\
     int j =0, k=0;\
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){\
-      smpi_comm_init_smp(comm);\
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){\
+      comm->init_smp();\
     }\
     int local_size=1;\
     }\
     int local_size=1;\
-    if (smpi_comm_is_uniform(comm)) {\
-        local_size = smpi_comm_size(smpi_comm_get_intra_comm(comm));\
+    if (comm->is_uniform()) {\
+        local_size = comm->get_intra_comm()->size();\
     }\
     while(i < INTEL_MAX_NB_PPN &&\
     local_size!=intel_ ## cat ## _table[i].ppn)\
     }\
     while(i < INTEL_MAX_NB_PPN &&\
     local_size!=intel_ ## cat ## _table[i].ppn)\
index 7ab6a1b..7e6aa52 100644 (file)
@@ -60,7 +60,7 @@ int smpi_coll_tuned_allreduce_mpich(void *sbuf, void *rbuf, int count,
                         MPI_Datatype dtype, MPI_Op op, MPI_Comm comm)
 {
     size_t dsize, block_dsize;
                         MPI_Datatype dtype, MPI_Op op, MPI_Comm comm)
 {
     size_t dsize, block_dsize;
-    int comm_size = smpi_comm_size(comm);
+    int comm_size = comm->size();
     const size_t large_message = 2048; //MPIR_PARAM_ALLREDUCE_SHORT_MSG_SIZE
 
     dsize = smpi_datatype_size(dtype);
     const size_t large_message = 2048; //MPIR_PARAM_ALLREDUCE_SHORT_MSG_SIZE
 
     dsize = smpi_datatype_size(dtype);
@@ -140,7 +140,7 @@ int smpi_coll_tuned_alltoall_mpich( void *sbuf, int scount,
 {
     int communicator_size;
     size_t dsize, block_dsize;
 {
     int communicator_size;
     size_t dsize, block_dsize;
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     unsigned int short_size=256;
     unsigned int medium_size=32768;
 
     unsigned int short_size=256;
     unsigned int medium_size=32768;
@@ -257,7 +257,7 @@ int smpi_coll_tuned_bcast_mpich(void *buff, int count,
     //int segsize = 0;
     size_t message_size, dsize;
 
     //int segsize = 0;
     size_t message_size, dsize;
 
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* else we need data size for decision function */
     dsize = smpi_datatype_size(datatype);
 
     /* else we need data size for decision function */
     dsize = smpi_datatype_size(datatype);
@@ -348,7 +348,7 @@ int smpi_coll_tuned_reduce_mpich( void *sendbuf, void *recvbuf,
     int communicator_size=0;
     //int segsize = 0;
     size_t message_size, dsize;
     int communicator_size=0;
     //int segsize = 0;
     size_t message_size, dsize;
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* need data size for decision function */
     dsize=smpi_datatype_size(datatype);
 
     /* need data size for decision function */
     dsize=smpi_datatype_size(datatype);
@@ -430,7 +430,7 @@ int smpi_coll_tuned_reduce_scatter_mpich( void *sbuf, void *rbuf,
 
     XBT_DEBUG("smpi_coll_tuned_reduce_scatter_mpich");
     
 
     XBT_DEBUG("smpi_coll_tuned_reduce_scatter_mpich");
     
-    comm_size = smpi_comm_size(comm);
+    comm_size = comm->size();
     // We need data size for decision function 
     total_message_size = 0;
     for (i = 0; i < comm_size; i++) { 
     // We need data size for decision function 
     total_message_size = 0;
     for (i = 0; i < comm_size; i++) { 
@@ -521,7 +521,7 @@ int smpi_coll_tuned_allgather_mpich(void *sbuf, int scount,
     int communicator_size, pow2_size;
     size_t dsize, total_dsize;
 
     int communicator_size, pow2_size;
     size_t dsize, total_dsize;
 
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* Determine complete data size */
     dsize=smpi_datatype_size(sdtype);
 
     /* Determine complete data size */
     dsize=smpi_datatype_size(sdtype);
@@ -600,7 +600,7 @@ int smpi_coll_tuned_allgatherv_mpich(void *sbuf, int scount,
     int communicator_size, pow2_size,i;
     size_t total_dsize;
 
     int communicator_size, pow2_size,i;
     size_t total_dsize;
 
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* Determine complete data size */
     total_dsize = 0;
 
     /* Determine complete data size */
     total_dsize = 0;
@@ -689,7 +689,7 @@ int smpi_coll_tuned_scatter_mpich(void *sbuf, int scount,
                                             int root, MPI_Comm  comm
                                             )
 {
                                             int root, MPI_Comm  comm
                                             )
 {
-  if(smpi_comm_rank(comm)!=root){
+  if(comm->rank()!=root){
       sbuf=xbt_malloc(rcount*smpi_datatype_get_extent(rdtype));
       scount=rcount;
       sdtype=rdtype;
       sbuf=xbt_malloc(rcount*smpi_datatype_get_extent(rdtype));
       scount=rcount;
       sdtype=rdtype;
@@ -697,7 +697,7 @@ int smpi_coll_tuned_scatter_mpich(void *sbuf, int scount,
   int ret= smpi_coll_tuned_scatter_ompi_binomial (sbuf, scount, sdtype,
                                                        rbuf, rcount, rdtype, 
                                                        root, comm);
   int ret= smpi_coll_tuned_scatter_ompi_binomial (sbuf, scount, sdtype,
                                                        rbuf, rcount, rdtype, 
                                                        root, comm);
-  if(smpi_comm_rank(comm)!=root){
+  if(comm->rank()!=root){
       xbt_free(sbuf);
   }
   return ret;
       xbt_free(sbuf);
   }
   return ret;
index 54d259d..4dc8c57 100644 (file)
@@ -28,7 +28,7 @@ int smpi_coll_tuned_alltoall_mvapich2( void *sendbuf, int sendcount,
   int range = 0;
   int range_threshold = 0;
   int conf_index = 0;
   int range = 0;
   int range_threshold = 0;
   int conf_index = 0;
-  comm_size =  smpi_comm_size(comm);
+  comm_size =  comm->size();
 
   sendtype_size=smpi_datatype_size(sendtype);
   recvtype_size=smpi_datatype_size(recvtype);
 
   sendtype_size=smpi_datatype_size(sendtype);
   recvtype_size=smpi_datatype_size(recvtype);
@@ -98,21 +98,21 @@ int smpi_coll_tuned_allgather_mvapich2(void *sendbuf, int sendcount, MPI_Datatyp
   MPI_Comm shmem_comm;
   //MPI_Comm *shmem_commptr=NULL;
   /* Get the size of the communicator */
   MPI_Comm shmem_comm;
   //MPI_Comm *shmem_commptr=NULL;
   /* Get the size of the communicator */
-  comm_size = smpi_comm_size(comm);
+  comm_size = comm->size();
   recvtype_size=smpi_datatype_size(recvtype);
   nbytes = recvtype_size * recvcount;
 
   if(mv2_allgather_table_ppn_conf==NULL)
     init_mv2_allgather_tables_stampede();
     
   recvtype_size=smpi_datatype_size(recvtype);
   nbytes = recvtype_size * recvcount;
 
   if(mv2_allgather_table_ppn_conf==NULL)
     init_mv2_allgather_tables_stampede();
     
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
 
   int i;
   }
 
   int i;
-  if (smpi_comm_is_uniform(comm)){
-    shmem_comm = smpi_comm_get_intra_comm(comm);
-    local_size = smpi_comm_size(shmem_comm);
+  if (comm->is_uniform()){
+    shmem_comm = comm->get_intra_comm();
+    local_size = shmem_comm->size();
     i = 0;
     if (mv2_allgather_table_ppn_conf[0] == -1) {
       // Indicating user defined tuning
     i = 0;
     if (mv2_allgather_table_ppn_conf[0] == -1) {
       // Indicating user defined tuning
@@ -158,7 +158,7 @@ int smpi_coll_tuned_allgather_mvapich2(void *sendbuf, int sendcount, MPI_Datatyp
   /* intracommunicator */
   if(is_two_level ==1){
     if(partial_sub_ok ==1){
   /* intracommunicator */
   if(is_two_level ==1){
     if(partial_sub_ok ==1){
-      if (smpi_comm_is_blocked(comm)){
+      if (comm->is_blocked()){
       mpi_errno = MPIR_2lvl_Allgather_MV2(sendbuf, sendcount, sendtype,
                             recvbuf, recvcount, recvtype,
                             comm);
       mpi_errno = MPIR_2lvl_Allgather_MV2(sendbuf, sendcount, sendtype,
                             recvbuf, recvcount, recvtype,
                             comm);
@@ -205,8 +205,8 @@ int smpi_coll_tuned_gather_mvapich2(void *sendbuf,
   int comm_size = 0;
   int recvtype_size, sendtype_size;
   int rank = -1;
   int comm_size = 0;
   int recvtype_size, sendtype_size;
   int rank = -1;
-  comm_size = smpi_comm_size(comm);
-  rank = smpi_comm_rank(comm);
+  comm_size = comm->size();
+  rank = comm->rank();
 
   if (rank == root) {
       recvtype_size=smpi_datatype_size(recvtype);
 
   if (rank == root) {
       recvtype_size=smpi_datatype_size(recvtype);
@@ -239,7 +239,7 @@ int smpi_coll_tuned_gather_mvapich2(void *sendbuf,
       range_intra_threshold++;
   }
   
       range_intra_threshold++;
   }
   
-    if (smpi_comm_is_blocked(comm) ) {
+    if (comm->is_blocked() ) {
         // Set intra-node function pt for gather_two_level 
         MV2_Gather_intra_node_function = 
                               mv2_gather_thresholds_table[range].intra_node[range_intra_threshold].
         // Set intra-node function pt for gather_two_level 
         MV2_Gather_intra_node_function = 
                               mv2_gather_thresholds_table[range].intra_node[range_intra_threshold].
@@ -276,7 +276,7 @@ int smpi_coll_tuned_allgatherv_mvapich2(void *sendbuf, int sendcount, MPI_Dataty
   if(mv2_allgatherv_thresholds_table==NULL)
     init_mv2_allgatherv_tables_stampede();
 
   if(mv2_allgatherv_thresholds_table==NULL)
     init_mv2_allgatherv_tables_stampede();
 
-  comm_size = smpi_comm_size(comm);
+  comm_size = comm->size();
   total_count = 0;
   for (i = 0; i < comm_size; i++)
     total_count += recvcounts[i];
   total_count = 0;
   for (i = 0; i < comm_size; i++)
     total_count += recvcounts[i];
@@ -341,8 +341,8 @@ int smpi_coll_tuned_allreduce_mvapich2(void *sendbuf,
   //int rank = 0,
   int comm_size = 0;
 
   //int rank = 0,
   int comm_size = 0;
 
-  comm_size = smpi_comm_size(comm);
-  //rank = smpi_comm_rank(comm);
+  comm_size = comm->size();
+  //rank = comm->rank();
 
   if (count == 0) {
       return MPI_SUCCESS;
 
   if (count == 0) {
       return MPI_SUCCESS;
@@ -426,8 +426,8 @@ int smpi_coll_tuned_allreduce_mvapich2(void *sendbuf,
     if(is_two_level == 1){
         // check if shm is ready, if not use other algorithm first
         if (is_commutative) {
     if(is_two_level == 1){
         // check if shm is ready, if not use other algorithm first
         if (is_commutative) {
-          if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-            smpi_comm_init_smp(comm);
+          if(comm->get_leaders_comm()==MPI_COMM_NULL){
+            comm->init_smp();
           }
           mpi_errno = MPIR_Allreduce_two_level_MV2(sendbuf, recvbuf, count,
                                                      datatype, op, comm);
           }
           mpi_errno = MPIR_Allreduce_two_level_MV2(sendbuf, recvbuf, count,
                                                      datatype, op, comm);
@@ -497,13 +497,13 @@ int smpi_coll_tuned_bcast_mvapich2(void *buffer,
 
     if (count == 0)
         return MPI_SUCCESS;
 
     if (count == 0)
         return MPI_SUCCESS;
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      smpi_comm_init_smp(comm);
+    if(comm->get_leaders_comm()==MPI_COMM_NULL){
+      comm->init_smp();
     }
     if(!mv2_bcast_thresholds_table)
       init_mv2_bcast_tables_stampede();
     }
     if(!mv2_bcast_thresholds_table)
       init_mv2_bcast_tables_stampede();
-    comm_size = smpi_comm_size(comm);
-    //rank = smpi_comm_rank(comm);
+    comm_size = comm->size();
+    //rank = comm->rank();
 
     is_contig=1;
 /*    if (HANDLE_GET_KIND(datatype) == HANDLE_KIND_BUILTIN)*/
 
     is_contig=1;
 /*    if (HANDLE_GET_KIND(datatype) == HANDLE_KIND_BUILTIN)*/
@@ -625,7 +625,7 @@ int smpi_coll_tuned_bcast_mvapich2(void *buffer,
         } else 
 #endif /* defined(CHANNEL_MRAIL_GEN2) */
         { 
         } else 
 #endif /* defined(CHANNEL_MRAIL_GEN2) */
         { 
-            shmem_comm = smpi_comm_get_intra_comm(comm);
+            shmem_comm = comm->get_intra_comm();
             if (!is_contig || !is_homogeneous) {
                 mpi_errno =
                     MPIR_Bcast_tune_inter_node_helper_MV2(tmp_buf, nbytes, MPI_BYTE,
             if (!is_contig || !is_homogeneous) {
                 mpi_errno =
                     MPIR_Bcast_tune_inter_node_helper_MV2(tmp_buf, nbytes, MPI_BYTE,
@@ -697,7 +697,7 @@ int smpi_coll_tuned_reduce_mvapich2( void *sendbuf,
   int sendtype_size;
   int is_two_level = 0;
 
   int sendtype_size;
   int is_two_level = 0;
 
-  comm_size = smpi_comm_size(comm);
+  comm_size = comm->size();
   sendtype_size=smpi_datatype_size(datatype);
   nbytes = count * sendtype_size;
 
   sendtype_size=smpi_datatype_size(datatype);
   nbytes = count * sendtype_size;
 
@@ -758,8 +758,8 @@ int smpi_coll_tuned_reduce_mvapich2( void *sendbuf,
   if(is_two_level == 1)
     {
        if (is_commutative == 1) {
   if(is_two_level == 1)
     {
        if (is_commutative == 1) {
-         if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-           smpi_comm_init_smp(comm);
+         if(comm->get_leaders_comm()==MPI_COMM_NULL){
+           comm->init_smp();
          }
          mpi_errno = MPIR_Reduce_two_level_helper_MV2(sendbuf, recvbuf, count, 
                                            datatype, op, root, comm);
          }
          mpi_errno = MPIR_Reduce_two_level_helper_MV2(sendbuf, recvbuf, count, 
                                            datatype, op, root, comm);
@@ -801,7 +801,7 @@ int smpi_coll_tuned_reduce_scatter_mvapich2(void *sendbuf, void *recvbuf, int *r
     MPI_Comm comm)
 {
   int mpi_errno = MPI_SUCCESS;
     MPI_Comm comm)
 {
   int mpi_errno = MPI_SUCCESS;
-  int i = 0, comm_size = smpi_comm_size(comm), total_count = 0, type_size =
+  int i = 0, comm_size = comm->size(), total_count = 0, type_size =
       0, nbytes = 0;
   int range = 0;
   int range_threshold = 0;
       0, nbytes = 0;
   int range = 0;
   int range_threshold = 0;
@@ -893,13 +893,13 @@ int smpi_coll_tuned_scatter_mvapich2(void *sendbuf,
   if(mv2_scatter_thresholds_table==NULL)
     init_mv2_scatter_tables_stampede();
 
   if(mv2_scatter_thresholds_table==NULL)
     init_mv2_scatter_tables_stampede();
 
-  if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-    smpi_comm_init_smp(comm);
+  if(comm->get_leaders_comm()==MPI_COMM_NULL){
+    comm->init_smp();
   }
   
   }
   
-  comm_size = smpi_comm_size(comm);
+  comm_size = comm->size();
 
 
-  rank = smpi_comm_rank(comm);
+  rank = comm->rank();
 
   if (rank == root) {
       sendtype_size=smpi_datatype_size(sendtype);
 
   if (rank == root) {
       sendtype_size=smpi_datatype_size(sendtype);
@@ -910,10 +910,10 @@ int smpi_coll_tuned_scatter_mvapich2(void *sendbuf,
   }
   
     // check if safe to use partial subscription mode 
   }
   
     // check if safe to use partial subscription mode 
-    if (smpi_comm_is_uniform(comm)) {
+    if (comm->is_uniform()) {
 
 
-        shmem_comm = smpi_comm_get_intra_comm(comm);
-        local_size = smpi_comm_size(shmem_comm);
+        shmem_comm = comm->get_intra_comm();
+        local_size = shmem_comm->size();
         i = 0;
         if (mv2_scatter_table_ppn_conf[0] == -1) {
             // Indicating user defined tuning 
         i = 0;
         if (mv2_scatter_table_ppn_conf[0] == -1) {
             // Indicating user defined tuning 
@@ -982,7 +982,7 @@ int smpi_coll_tuned_scatter_mvapich2(void *sendbuf,
 
   if( (MV2_Scatter_function == &MPIR_Scatter_MV2_two_level_Direct) ||
       (MV2_Scatter_function == &MPIR_Scatter_MV2_two_level_Binomial)) {
 
   if( (MV2_Scatter_function == &MPIR_Scatter_MV2_two_level_Direct) ||
       (MV2_Scatter_function == &MPIR_Scatter_MV2_two_level_Binomial)) {
-       if( smpi_comm_is_blocked(comm)) {
+       if( comm->is_blocked()) {
              MV2_Scatter_intra_function = mv2_scatter_thresholds_table[conf_index][range].intra_node[range_threshold_intra]
                                 .MV2_pt_Scatter_function;
 
              MV2_Scatter_intra_function = mv2_scatter_thresholds_table[conf_index][range].intra_node[range_threshold_intra]
                                 .MV2_pt_Scatter_function;
 
index b0fb666..a0acf0c 100644 (file)
@@ -13,7 +13,7 @@ int smpi_coll_tuned_allreduce_ompi(void *sbuf, void *rbuf, int count,
                         MPI_Datatype dtype, MPI_Op op, MPI_Comm comm)
 {
     size_t dsize, block_dsize;
                         MPI_Datatype dtype, MPI_Op op, MPI_Comm comm)
 {
     size_t dsize, block_dsize;
-    int comm_size = smpi_comm_size(comm);
+    int comm_size = comm->size();
     const size_t intermediate_message = 10000;
 
     /**
     const size_t intermediate_message = 10000;
 
     /**
@@ -61,7 +61,7 @@ int smpi_coll_tuned_alltoall_ompi( void *sbuf, int scount,
 {
     int communicator_size;
     size_t dsize, block_dsize;
 {
     int communicator_size;
     size_t dsize, block_dsize;
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* Decision function based on measurement on Grig cluster at 
        the University of Tennessee (2GB MX) up to 64 nodes.
 
     /* Decision function based on measurement on Grig cluster at 
        the University of Tennessee (2GB MX) up to 64 nodes.
@@ -101,7 +101,7 @@ int smpi_coll_tuned_alltoallv_ompi(void *sbuf, int *scounts, int *sdisps,
 
 
 int smpi_coll_tuned_barrier_ompi(MPI_Comm  comm)
 
 
 int smpi_coll_tuned_barrier_ompi(MPI_Comm  comm)
-{    int communicator_size = smpi_comm_size(comm);
+{    int communicator_size = comm->size();
 
     if( 2 == communicator_size )
         return smpi_coll_tuned_barrier_ompi_two_procs(comm);
 
     if( 2 == communicator_size )
         return smpi_coll_tuned_barrier_ompi_two_procs(comm);
@@ -141,7 +141,7 @@ int smpi_coll_tuned_bcast_ompi(void *buff, int count,
     //int segsize = 0;
     size_t message_size, dsize;
 
     //int segsize = 0;
     size_t message_size, dsize;
 
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* else we need data size for decision function */
     dsize = smpi_datatype_size(datatype);
 
     /* else we need data size for decision function */
     dsize = smpi_datatype_size(datatype);
@@ -243,7 +243,7 @@ int smpi_coll_tuned_reduce_ompi( void *sendbuf, void *recvbuf,
     /* no limit on # of outstanding requests */
     //const int max_requests = 0;
 
     /* no limit on # of outstanding requests */
     //const int max_requests = 0;
 
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* need data size for decision function */
     dsize=smpi_datatype_size(datatype);
 
     /* need data size for decision function */
     dsize=smpi_datatype_size(datatype);
@@ -342,7 +342,7 @@ int smpi_coll_tuned_reduce_scatter_ompi( void *sbuf, void *rbuf,
 
     XBT_DEBUG("smpi_coll_tuned_reduce_scatter_ompi");
     
 
     XBT_DEBUG("smpi_coll_tuned_reduce_scatter_ompi");
     
-    comm_size = smpi_comm_size(comm);
+    comm_size = comm->size();
     // We need data size for decision function 
     dsize=smpi_datatype_size(dtype);
     total_message_size = 0;
     // We need data size for decision function 
     dsize=smpi_datatype_size(dtype);
     total_message_size = 0;
@@ -391,7 +391,7 @@ int smpi_coll_tuned_allgather_ompi(void *sbuf, int scount,
     int communicator_size, pow2_size;
     size_t dsize, total_dsize;
 
     int communicator_size, pow2_size;
     size_t dsize, total_dsize;
 
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
 
     /* Special case for 2 processes */
     if (communicator_size == 2) {
 
     /* Special case for 2 processes */
     if (communicator_size == 2) {
@@ -473,7 +473,7 @@ int smpi_coll_tuned_allgatherv_ompi(void *sbuf, int scount,
     int communicator_size;
     size_t dsize, total_dsize;
     
     int communicator_size;
     size_t dsize, total_dsize;
     
-    communicator_size = smpi_comm_size(comm);
+    communicator_size = comm->size();
     
     /* Special case for 2 processes */
     if (communicator_size == 2) {
     
     /* Special case for 2 processes */
     if (communicator_size == 2) {
@@ -534,8 +534,8 @@ int smpi_coll_tuned_gather_ompi(void *sbuf, int scount,
 
     XBT_DEBUG("smpi_coll_tuned_gather_ompi");
 
 
     XBT_DEBUG("smpi_coll_tuned_gather_ompi");
 
-    communicator_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    communicator_size = comm->size();
+    rank = comm->rank();
 
     // Determine block size 
     if (rank == root) {
 
     // Determine block size 
     if (rank == root) {
@@ -584,8 +584,8 @@ int smpi_coll_tuned_scatter_ompi(void *sbuf, int scount,
 
     XBT_DEBUG("smpi_coll_tuned_scatter_ompi");
 
 
     XBT_DEBUG("smpi_coll_tuned_scatter_ompi");
 
-    communicator_size = smpi_comm_size(comm);
-    rank = smpi_comm_rank(comm);
+    communicator_size = comm->size();
+    rank = comm->rank();
     // Determine block size 
     if (root == rank) {
         dsize=smpi_datatype_size(sdtype);
     // Determine block size 
     if (root == rank) {
         dsize=smpi_datatype_size(sdtype);
index 52b5803..d600eb7 100644 (file)
@@ -9,6 +9,8 @@
 
 #include "simgrid/simix.h"
 #include "smpi/smpi.h"
 
 #include "simgrid/simix.h"
 #include "smpi/smpi.h"
+#include "src/smpi/smpi_group.hpp"
+#include "src/smpi/smpi_comm.hpp"
 #include "src/include/smpi/smpi_interface.h"
 #include "src/instr/instr_private.h"
 #include "src/internal_config.h"
 #include "src/include/smpi/smpi_interface.h"
 #include "src/instr/instr_private.h"
 #include "src/internal_config.h"
@@ -186,7 +188,7 @@ XBT_PRIVATE bool smpi_process_get_replaying();
 XBT_PRIVATE void smpi_deployment_register_process(const char* instance_id, int rank, int index, MPI_Comm** comm,
                                                   msg_bar_t* bar);
 XBT_PRIVATE void smpi_deployment_cleanup_instances();
 XBT_PRIVATE void smpi_deployment_register_process(const char* instance_id, int rank, int index, MPI_Comm** comm,
                                                   msg_bar_t* bar);
 XBT_PRIVATE void smpi_deployment_cleanup_instances();
-
 XBT_PRIVATE void smpi_comm_copy_buffer_callback(smx_activity_t comm, void *buff, size_t buff_size);
 
 XBT_PRIVATE void smpi_comm_null_copy_buffer_callback(smx_activity_t comm, void *buff, size_t buff_size);
 XBT_PRIVATE void smpi_comm_copy_buffer_callback(smx_activity_t comm, void *buff, size_t buff_size);
 
 XBT_PRIVATE void smpi_comm_null_copy_buffer_callback(smx_activity_t comm, void *buff, size_t buff_size);
@@ -246,29 +248,6 @@ XBT_PRIVATE void smpi_op_set_fortran(MPI_Op op);
 XBT_PRIVATE void smpi_op_apply(MPI_Op op, void *invec, void *inoutvec, int *len, MPI_Datatype * datatype);
 
 
 XBT_PRIVATE void smpi_op_apply(MPI_Op op, void *invec, void *inoutvec, int *len, MPI_Datatype * datatype);
 
 
-XBT_PRIVATE MPI_Topology smpi_comm_topo(MPI_Comm comm);
-XBT_PRIVATE MPI_Comm smpi_comm_new(MPI_Group group, MPI_Topology topo);
-XBT_PRIVATE void smpi_comm_destroy(MPI_Comm comm);
-XBT_PRIVATE MPI_Group smpi_comm_group(MPI_Comm comm);
-XBT_PRIVATE int smpi_comm_size(MPI_Comm comm);
-XBT_PRIVATE void smpi_comm_get_name(MPI_Comm comm, char* name, int* len);
-XBT_PRIVATE int smpi_comm_rank(MPI_Comm comm);
-XBT_PRIVATE MPI_Comm smpi_comm_split(MPI_Comm comm, int color, int key);
-XBT_PRIVATE int smpi_comm_dup(MPI_Comm comm, MPI_Comm* newcomm);
-XBT_PRIVATE void smpi_comm_use(MPI_Comm comm);
-XBT_PRIVATE void smpi_comm_unuse(MPI_Comm comm);
-XBT_PRIVATE void smpi_comm_cleanup_attributes(MPI_Comm comm);
-XBT_PRIVATE void smpi_comm_cleanup_smp(MPI_Comm comm);
-XBT_PRIVATE void smpi_comm_set_leaders_comm(MPI_Comm comm, MPI_Comm leaders);
-XBT_PRIVATE void smpi_comm_set_intra_comm(MPI_Comm comm, MPI_Comm leaders);
-XBT_PRIVATE int* smpi_comm_get_non_uniform_map(MPI_Comm comm);
-XBT_PRIVATE int* smpi_comm_get_leaders_map(MPI_Comm comm);
-XBT_PRIVATE MPI_Comm smpi_comm_get_leaders_comm(MPI_Comm comm);
-XBT_PRIVATE MPI_Comm smpi_comm_get_intra_comm(MPI_Comm comm);
-XBT_PRIVATE int smpi_comm_is_uniform(MPI_Comm comm);
-XBT_PRIVATE int smpi_comm_is_blocked(MPI_Comm comm);
-XBT_PRIVATE void smpi_comm_init_smp(MPI_Comm comm);
-
 XBT_PRIVATE int smpi_comm_c2f(MPI_Comm comm);
 XBT_PRIVATE int smpi_comm_add_f(MPI_Comm comm);
 XBT_PRIVATE MPI_Comm smpi_comm_f2c(int comm);
 XBT_PRIVATE int smpi_comm_c2f(MPI_Comm comm);
 XBT_PRIVATE int smpi_comm_add_f(MPI_Comm comm);
 XBT_PRIVATE MPI_Comm smpi_comm_f2c(int comm);
@@ -385,13 +364,10 @@ XBT_PRIVATE int smpi_coll_tuned_alltoall_basic_linear(void *sendbuf, int sendcou
                                           void *recvbuf, int recvcount, MPI_Datatype recvtype, MPI_Comm comm);
 XBT_PRIVATE int smpi_coll_basic_alltoallv(void *sendbuf, int *sendcounts, int *senddisps, MPI_Datatype sendtype,
                               void *recvbuf, int *recvcounts, int *recvdisps, MPI_Datatype recvtype, MPI_Comm comm);
                                           void *recvbuf, int recvcount, MPI_Datatype recvtype, MPI_Comm comm);
 XBT_PRIVATE int smpi_coll_basic_alltoallv(void *sendbuf, int *sendcounts, int *senddisps, MPI_Datatype sendtype,
                               void *recvbuf, int *recvcounts, int *recvdisps, MPI_Datatype recvtype, MPI_Comm comm);
-
 XBT_PRIVATE int smpi_comm_keyval_create(MPI_Comm_copy_attr_function* copy_fn, MPI_Comm_delete_attr_function* delete_fn,
                                         int* keyval, void* extra_state);
 XBT_PRIVATE int smpi_comm_keyval_free(int* keyval);
 XBT_PRIVATE int smpi_comm_keyval_create(MPI_Comm_copy_attr_function* copy_fn, MPI_Comm_delete_attr_function* delete_fn,
                                         int* keyval, void* extra_state);
 XBT_PRIVATE int smpi_comm_keyval_free(int* keyval);
-XBT_PRIVATE int smpi_comm_attr_get(MPI_Comm comm, int keyval, void* attr_value, int* flag);
-XBT_PRIVATE int smpi_comm_attr_delete(MPI_Comm comm, int keyval);
-XBT_PRIVATE int smpi_comm_attr_put(MPI_Comm comm, int keyval, void* attr_value);
+
 XBT_PRIVATE int smpi_type_attr_delete(MPI_Datatype type, int keyval);
 XBT_PRIVATE int smpi_type_attr_get(MPI_Datatype type, int keyval, void* attr_value, int* flag);
 XBT_PRIVATE int smpi_type_attr_put(MPI_Datatype type, int keyval, void* attr_value);
 XBT_PRIVATE int smpi_type_attr_delete(MPI_Datatype type, int keyval);
 XBT_PRIVATE int smpi_type_attr_get(MPI_Datatype type, int keyval, void* attr_value, int* flag);
 XBT_PRIVATE int smpi_type_attr_put(MPI_Datatype type, int keyval, void* attr_value);
@@ -423,9 +399,6 @@ XBT_PRIVATE void* smpi_get_tmp_sendbuffer(int size);
 XBT_PRIVATE void* smpi_get_tmp_recvbuffer(int size);
 XBT_PRIVATE void  smpi_free_tmp_buffer(void* buf);
 
 XBT_PRIVATE void* smpi_get_tmp_recvbuffer(int size);
 XBT_PRIVATE void  smpi_free_tmp_buffer(void* buf);
 
-XBT_PRIVATE int smpi_comm_attr_get(MPI_Comm comm, int keyval, void* attr_value, int* flag);
-XBT_PRIVATE XBT_PRIVATE int smpi_comm_attr_delete(MPI_Comm comm, int keyval);
-XBT_PRIVATE int smpi_comm_attr_put(MPI_Comm comm, int keyval, void* attr_value);
 
 // f77 wrappers
 void mpi_init_(int* ierr);
 
 // f77 wrappers
 void mpi_init_(int* ierr);
index 8ca91f9..6e4ca69 100644 (file)
@@ -219,7 +219,7 @@ static MPI_Request build_request(void *buf, int count, MPI_Datatype datatype, in
   request->dst  = dst;
   request->tag  = tag;
   request->comm = comm;
   request->dst  = dst;
   request->tag  = tag;
   request->comm = comm;
-  smpi_comm_use(request->comm);
+  request->comm->use();
   request->action          = nullptr;
   request->flags           = flags;
   request->detached        = 0;
   request->action          = nullptr;
   request->flags           = flags;
   request->detached        = 0;
@@ -260,7 +260,7 @@ MPI_Request smpi_mpi_send_init(void *buf, int count, MPI_Datatype datatype, int
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
-                          smpi_comm_group(comm)->index(dst), tag, comm, PERSISTENT | SEND | PREPARED);
+                          comm->group()->index(dst), tag, comm, PERSISTENT | SEND | PREPARED);
   return request;
 }
 
   return request;
 }
 
@@ -268,7 +268,7 @@ MPI_Request smpi_mpi_ssend_init(void *buf, int count, MPI_Datatype datatype, int
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
-                        smpi_comm_group(comm)->index(dst), tag, comm, PERSISTENT | SSEND | SEND | PREPARED);
+                        comm->group()->index(dst), tag, comm, PERSISTENT | SSEND | SEND | PREPARED);
   return request;
 }
 
   return request;
 }
 
@@ -276,7 +276,7 @@ MPI_Request smpi_mpi_recv_init(void *buf, int count, MPI_Datatype datatype, int
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype,
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype,
-                          src == MPI_ANY_SOURCE ? MPI_ANY_SOURCE : smpi_comm_group(comm)->index(src),
+                          src == MPI_ANY_SOURCE ? MPI_ANY_SOURCE : comm->group()->index(src),
                           smpi_process_index(), tag, comm, PERSISTENT | RECV | PREPARED);
   return request;
 }
                           smpi_process_index(), tag, comm, PERSISTENT | RECV | PREPARED);
   return request;
 }
@@ -460,7 +460,7 @@ void smpi_mpi_request_free(MPI_Request * request)
 
     if((*request)->refcount==0){
         smpi_datatype_unuse((*request)->old_type);
 
     if((*request)->refcount==0){
         smpi_datatype_unuse((*request)->old_type);
-        smpi_comm_unuse((*request)->comm);
+        (*request)->comm->unuse();
         print_request("Destroying", (*request));
         xbt_free(*request);
         *request = MPI_REQUEST_NULL;
         print_request("Destroying", (*request));
         xbt_free(*request);
         *request = MPI_REQUEST_NULL;
@@ -506,7 +506,7 @@ MPI_Request smpi_isend_init(void *buf, int count, MPI_Datatype datatype, int dst
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf , count, datatype, smpi_process_index(),
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf , count, datatype, smpi_process_index(),
-                          smpi_comm_group(comm)->index(dst), tag,comm, PERSISTENT | ISEND | SEND | PREPARED);
+                          comm->group()->index(dst), tag,comm, PERSISTENT | ISEND | SEND | PREPARED);
   return request;
 }
 
   return request;
 }
 
@@ -514,7 +514,7 @@ MPI_Request smpi_mpi_isend(void *buf, int count, MPI_Datatype datatype, int dst,
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request =  build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request =  build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
-                           smpi_comm_group(comm)->index(dst), tag, comm, NON_PERSISTENT | ISEND | SEND);
+                           comm->group()->index(dst), tag, comm, NON_PERSISTENT | ISEND | SEND);
   smpi_mpi_start(request);
   return request;
 }
   smpi_mpi_start(request);
   return request;
 }
@@ -523,7 +523,7 @@ MPI_Request smpi_mpi_issend(void *buf, int count, MPI_Datatype datatype, int dst
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
-                        smpi_comm_group(comm)->index(dst), tag,comm, NON_PERSISTENT | ISEND | SSEND | SEND);
+                        comm->group()->index(dst), tag,comm, NON_PERSISTENT | ISEND | SSEND | SEND);
   smpi_mpi_start(request);
   return request;
 }
   smpi_mpi_start(request);
   return request;
 }
@@ -532,7 +532,7 @@ MPI_Request smpi_irecv_init(void *buf, int count, MPI_Datatype datatype, int src
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, src == MPI_ANY_SOURCE ? MPI_ANY_SOURCE :
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, src == MPI_ANY_SOURCE ? MPI_ANY_SOURCE :
-                          smpi_comm_group(comm)->index(src), smpi_process_index(), tag,
+                          comm->group()->index(src), smpi_process_index(), tag,
                           comm, PERSISTENT | RECV | PREPARED);
   return request;
 }
                           comm, PERSISTENT | RECV | PREPARED);
   return request;
 }
@@ -541,7 +541,7 @@ MPI_Request smpi_mpi_irecv(void *buf, int count, MPI_Datatype datatype, int src,
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, src == MPI_ANY_SOURCE ? MPI_ANY_SOURCE :
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, src == MPI_ANY_SOURCE ? MPI_ANY_SOURCE :
-                          smpi_comm_group(comm)->index(src), smpi_process_index(), tag, comm,
+                          comm->group()->index(src), smpi_process_index(), tag, comm,
                           NON_PERSISTENT | RECV);
   smpi_mpi_start(request);
   return request;
                           NON_PERSISTENT | RECV);
   smpi_mpi_start(request);
   return request;
@@ -559,7 +559,7 @@ void smpi_mpi_send(void *buf, int count, MPI_Datatype datatype, int dst, int tag
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
-                          smpi_comm_group(comm)->index(dst), tag, comm, NON_PERSISTENT | SEND);
+                          comm->group()->index(dst), tag, comm, NON_PERSISTENT | SEND);
 
   smpi_mpi_start(request);
   smpi_mpi_wait(&request, MPI_STATUS_IGNORE);
 
   smpi_mpi_start(request);
   smpi_mpi_wait(&request, MPI_STATUS_IGNORE);
@@ -570,7 +570,7 @@ void smpi_mpi_ssend(void *buf, int count, MPI_Datatype datatype, int dst, int ta
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
 {
   MPI_Request request = nullptr; /* MC needs the comm to be set to nullptr during the call */
   request = build_request(buf==MPI_BOTTOM ? nullptr : buf, count, datatype, smpi_process_index(),
-                          smpi_comm_group(comm)->index(dst), tag, comm, NON_PERSISTENT | SSEND | SEND);
+                          comm->group()->index(dst), tag, comm, NON_PERSISTENT | SSEND | SEND);
 
   smpi_mpi_start(request);
   smpi_mpi_wait(&request, MPI_STATUS_IGNORE);
 
   smpi_mpi_start(request);
   smpi_mpi_wait(&request, MPI_STATUS_IGNORE);
@@ -584,7 +584,7 @@ void smpi_mpi_sendrecv(void *sendbuf, int sendcount, MPI_Datatype sendtype,int d
   MPI_Request requests[2];
   MPI_Status stats[2];
   int myid=smpi_process_index();
   MPI_Request requests[2];
   MPI_Status stats[2];
   int myid=smpi_process_index();
-  if ((smpi_comm_group(comm)->index(dst) == myid) && (smpi_comm_group(comm)->index(src) == myid)){
+  if ((comm->group()->index(dst) == myid) && (comm->group()->index(src) == myid)){
       smpi_datatype_copy(sendbuf, sendcount, sendtype, recvbuf, recvcount, recvtype);
       return;
   }
       smpi_datatype_copy(sendbuf, sendcount, sendtype, recvbuf, recvcount, recvtype);
       return;
   }
@@ -613,7 +613,7 @@ static void finish_wait(MPI_Request * request, MPI_Status * status)
   if(!((req->detached != 0) && ((req->flags & SEND) != 0)) && ((req->flags & PREPARED) == 0)){
     if(status != MPI_STATUS_IGNORE) {
       int src = req->src == MPI_ANY_SOURCE ? req->real_src : req->src;
   if(!((req->detached != 0) && ((req->flags & SEND) != 0)) && ((req->flags & PREPARED) == 0)){
     if(status != MPI_STATUS_IGNORE) {
       int src = req->src == MPI_ANY_SOURCE ? req->real_src : req->src;
-      status->MPI_SOURCE = smpi_comm_group(req->comm)->rank(src);
+      status->MPI_SOURCE = req->comm->group()->rank(src);
       status->MPI_TAG = req->tag == MPI_ANY_TAG ? req->real_tag : req->tag;
       status->MPI_ERROR = req->truncated != 0 ? MPI_ERR_TRUNCATE : MPI_SUCCESS;
       // this handles the case were size in receive differs from size in send
       status->MPI_TAG = req->tag == MPI_ANY_TAG ? req->real_tag : req->tag;
       status->MPI_ERROR = req->truncated != 0 ? MPI_ERR_TRUNCATE : MPI_SUCCESS;
       // this handles the case were size in receive differs from size in send
@@ -776,7 +776,7 @@ void smpi_mpi_probe(int source, int tag, MPI_Comm comm, MPI_Status* status){
 
 void smpi_mpi_iprobe(int source, int tag, MPI_Comm comm, int* flag, MPI_Status* status){
   MPI_Request request = build_request(nullptr, 0, MPI_CHAR, source == MPI_ANY_SOURCE ? MPI_ANY_SOURCE :
 
 void smpi_mpi_iprobe(int source, int tag, MPI_Comm comm, int* flag, MPI_Status* status){
   MPI_Request request = build_request(nullptr, 0, MPI_CHAR, source == MPI_ANY_SOURCE ? MPI_ANY_SOURCE :
-                 smpi_comm_group(comm)->index(source), smpi_comm_rank(comm), tag, comm, PERSISTENT | RECV);
+                 comm->group()->index(source), comm->rank(), tag, comm, PERSISTENT | RECV);
 
   // to avoid deadlock, we have to sleep some time here, or the timer won't advance and we will only do iprobe simcalls
   // (especially when used as a break condition, such as while(MPI_Iprobe(...)) ... )
 
   // to avoid deadlock, we have to sleep some time here, or the timer won't advance and we will only do iprobe simcalls
   // (especially when used as a break condition, such as while(MPI_Iprobe(...)) ... )
@@ -812,7 +812,7 @@ void smpi_mpi_iprobe(int source, int tag, MPI_Comm comm, int* flag, MPI_Status*
     MPI_Request req                            = static_cast<MPI_Request>(sync_comm->src_data);
     *flag = 1;
     if(status != MPI_STATUS_IGNORE && (req->flags & PREPARED) == 0) {
     MPI_Request req                            = static_cast<MPI_Request>(sync_comm->src_data);
     *flag = 1;
     if(status != MPI_STATUS_IGNORE && (req->flags & PREPARED) == 0) {
-      status->MPI_SOURCE = smpi_comm_group(comm)->rank(req->src);
+      status->MPI_SOURCE = comm->group()->rank(req->src);
       status->MPI_TAG    = req->tag;
       status->MPI_ERROR  = MPI_SUCCESS;
       status->count      = req->real_size;
       status->MPI_TAG    = req->tag;
       status->MPI_ERROR  = MPI_SUCCESS;
       status->count      = req->real_size;
@@ -1030,8 +1030,8 @@ void smpi_mpi_gather(void *sendbuf, int sendcount, MPI_Datatype sendtype,
   MPI_Aint lb = 0;
   MPI_Aint recvext = 0;
 
   MPI_Aint lb = 0;
   MPI_Aint recvext = 0;
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   if(rank != root) {
     // Send buffer to root
     smpi_mpi_send(sendbuf, sendcount, sendtype, root, system_tag, comm);
   if(rank != root) {
     // Send buffer to root
     smpi_mpi_send(sendbuf, sendcount, sendtype, root, system_tag, comm);
@@ -1063,10 +1063,10 @@ void smpi_mpi_gather(void *sendbuf, int sendcount, MPI_Datatype sendtype,
 void smpi_mpi_reduce_scatter(void *sendbuf, void *recvbuf, int *recvcounts, MPI_Datatype datatype, MPI_Op op,
                              MPI_Comm comm)
 {
 void smpi_mpi_reduce_scatter(void *sendbuf, void *recvbuf, int *recvcounts, MPI_Datatype datatype, MPI_Op op,
                              MPI_Comm comm)
 {
-  int rank = smpi_comm_rank(comm);
+  int rank = comm->rank();
 
   /* arbitrarily choose root as rank 0 */
 
   /* arbitrarily choose root as rank 0 */
-  int size = smpi_comm_size(comm);
+  int size = comm->size();
   int count = 0;
   int *displs = xbt_new(int, size);
   for (int i = 0; i < size; i++) {
   int count = 0;
   int *displs = xbt_new(int, size);
   for (int i = 0; i < size; i++) {
@@ -1088,8 +1088,8 @@ void smpi_mpi_gatherv(void *sendbuf, int sendcount, MPI_Datatype sendtype, void
   MPI_Aint lb = 0;
   MPI_Aint recvext = 0;
 
   MPI_Aint lb = 0;
   MPI_Aint recvext = 0;
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   if (rank != root) {
     // Send buffer to root
     smpi_mpi_send(sendbuf, sendcount, sendtype, root, system_tag, comm);
   if (rank != root) {
     // Send buffer to root
     smpi_mpi_send(sendbuf, sendcount, sendtype, root, system_tag, comm);
@@ -1126,8 +1126,8 @@ void smpi_mpi_allgather(void *sendbuf, int sendcount, MPI_Datatype sendtype,
   MPI_Aint recvext = 0;
   MPI_Request *requests;
 
   MPI_Aint recvext = 0;
   MPI_Request *requests;
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   // FIXME: check for errors
   smpi_datatype_extent(recvtype, &lb, &recvext);
   // Local copy from self
   // FIXME: check for errors
   smpi_datatype_extent(recvtype, &lb, &recvext);
   // Local copy from self
@@ -1161,8 +1161,8 @@ void smpi_mpi_allgatherv(void *sendbuf, int sendcount, MPI_Datatype sendtype, vo
   MPI_Aint lb = 0;
   MPI_Aint recvext = 0;
 
   MPI_Aint lb = 0;
   MPI_Aint recvext = 0;
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   smpi_datatype_extent(recvtype, &lb, &recvext);
   // Local copy from self
   smpi_datatype_copy(sendbuf, sendcount, sendtype,
   smpi_datatype_extent(recvtype, &lb, &recvext);
   // Local copy from self
   smpi_datatype_copy(sendbuf, sendcount, sendtype,
@@ -1197,8 +1197,8 @@ void smpi_mpi_scatter(void *sendbuf, int sendcount, MPI_Datatype sendtype,
   MPI_Aint sendext = 0;
   MPI_Request *requests;
 
   MPI_Aint sendext = 0;
   MPI_Request *requests;
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   if(rank != root) {
     // Recv buffer from root
     smpi_mpi_recv(recvbuf, recvcount, recvtype, root, system_tag, comm, MPI_STATUS_IGNORE);
   if(rank != root) {
     // Recv buffer from root
     smpi_mpi_recv(recvbuf, recvcount, recvtype, root, system_tag, comm, MPI_STATUS_IGNORE);
@@ -1236,8 +1236,8 @@ void smpi_mpi_scatterv(void *sendbuf, int *sendcounts, int *displs, MPI_Datatype
   MPI_Aint lb = 0;
   MPI_Aint sendext = 0;
 
   MPI_Aint lb = 0;
   MPI_Aint sendext = 0;
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   if(rank != root) {
     // Recv buffer from root
     smpi_mpi_recv(recvbuf, recvcount, recvtype, root, system_tag, comm, MPI_STATUS_IGNORE);
   if(rank != root) {
     // Recv buffer from root
     smpi_mpi_recv(recvbuf, recvcount, recvtype, root, system_tag, comm, MPI_STATUS_IGNORE);
@@ -1277,8 +1277,8 @@ void smpi_mpi_reduce(void *sendbuf, void *recvbuf, int count, MPI_Datatype datat
 
   char* sendtmpbuf = static_cast<char *>(sendbuf);
 
 
   char* sendtmpbuf = static_cast<char *>(sendbuf);
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   //non commutative case, use a working algo from openmpi
   if(!smpi_op_is_commute(op)){
     smpi_coll_tuned_reduce_ompi_basic_linear(sendtmpbuf, recvbuf, count, datatype, op, root, comm);
   //non commutative case, use a working algo from openmpi
   if(!smpi_op_is_commute(op)){
     smpi_coll_tuned_reduce_ompi_basic_linear(sendtmpbuf, recvbuf, count, datatype, op, root, comm);
@@ -1350,8 +1350,8 @@ void smpi_mpi_scan(void *sendbuf, void *recvbuf, int count, MPI_Datatype datatyp
   MPI_Aint lb      = 0;
   MPI_Aint dataext = 0;
 
   MPI_Aint lb      = 0;
   MPI_Aint dataext = 0;
 
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
 
   smpi_datatype_extent(datatype, &lb, &dataext);
 
 
   smpi_datatype_extent(datatype, &lb, &dataext);
 
@@ -1410,8 +1410,8 @@ void smpi_mpi_exscan(void *sendbuf, void *recvbuf, int count, MPI_Datatype datat
   MPI_Aint lb         = 0;
   MPI_Aint dataext    = 0;
   int recvbuf_is_empty=1;
   MPI_Aint lb         = 0;
   MPI_Aint dataext    = 0;
   int recvbuf_is_empty=1;
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
 
   smpi_datatype_extent(datatype, &lb, &dataext);
 
 
   smpi_datatype_extent(datatype, &lb, &dataext);
 
index 7873514..db2431f 100644 (file)
@@ -340,7 +340,7 @@ static unsigned int private_sleep(double secs)
   smpi_bench_end();
 
   XBT_DEBUG("Sleep for: %lf secs", secs);
   smpi_bench_end();
 
   XBT_DEBUG("Sleep for: %lf secs", secs);
-  int rank = smpi_comm_rank(MPI_COMM_WORLD);
+  int rank = MPI_COMM_WORLD->rank();
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type=TRACING_SLEEPING;
   extra->sleep_duration=secs;
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type=TRACING_SLEEPING;
   extra->sleep_duration=secs;
index e6fae05..1fbc744 100644 (file)
@@ -125,7 +125,7 @@ void (*smpi_coll_cleanup_callback)();
 int smpi_coll_tuned_alltoall_ompi2(void *sendbuf, int sendcount, MPI_Datatype sendtype, void *recvbuf,
                                    int recvcount, MPI_Datatype recvtype, MPI_Comm comm)
 {
 int smpi_coll_tuned_alltoall_ompi2(void *sendbuf, int sendcount, MPI_Datatype sendtype, void *recvbuf,
                                    int recvcount, MPI_Datatype recvtype, MPI_Comm comm)
 {
-  int size = smpi_comm_size(comm);
+  int size = comm->size();
   int sendsize = smpi_datatype_size(sendtype) * sendcount;
   if (sendsize < 200 && size > 12) {
     return smpi_coll_tuned_alltoall_bruck(sendbuf, sendcount, sendtype, recvbuf, recvcount, recvtype, comm);
   int sendsize = smpi_datatype_size(sendtype) * sendcount;
   if (sendsize < 200 && size > 12) {
     return smpi_coll_tuned_alltoall_bruck(sendbuf, sendcount, sendtype, recvbuf, recvcount, recvtype, comm);
@@ -154,8 +154,8 @@ int smpi_coll_tuned_alltoall_bruck(void *sendbuf, int sendcount, MPI_Datatype se
   MPI_Request *requests;
 
   // FIXME: check implementation
   MPI_Request *requests;
 
   // FIXME: check implementation
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   XBT_DEBUG("<%d> algorithm alltoall_bruck() called.", rank);
   smpi_datatype_extent(sendtype, &lb, &sendext);
   smpi_datatype_extent(recvtype, &lb, &recvext);
   XBT_DEBUG("<%d> algorithm alltoall_bruck() called.", rank);
   smpi_datatype_extent(sendtype, &lb, &sendext);
   smpi_datatype_extent(recvtype, &lb, &recvext);
@@ -212,8 +212,8 @@ int smpi_coll_tuned_alltoall_basic_linear(void *sendbuf, int sendcount, MPI_Data
   MPI_Request *requests;
 
   /* Initialize. */
   MPI_Request *requests;
 
   /* Initialize. */
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   XBT_DEBUG("<%d> algorithm alltoall_basic_linear() called.", rank);
   smpi_datatype_extent(sendtype, &lb, &sendext);
   smpi_datatype_extent(recvtype, &lb, &recvext);
   XBT_DEBUG("<%d> algorithm alltoall_basic_linear() called.", rank);
   smpi_datatype_extent(sendtype, &lb, &sendext);
   smpi_datatype_extent(recvtype, &lb, &recvext);
@@ -265,8 +265,8 @@ int smpi_coll_basic_alltoallv(void *sendbuf, int *sendcounts, int *senddisps, MP
   MPI_Request *requests;
 
   /* Initialize. */
   MPI_Request *requests;
 
   /* Initialize. */
-  int rank = smpi_comm_rank(comm);
-  int size = smpi_comm_size(comm);
+  int rank = comm->rank();
+  int size = comm->size();
   XBT_DEBUG("<%d> algorithm basic_alltoallv() called.", rank);
   smpi_datatype_extent(sendtype, &lb, &sendext);
   smpi_datatype_extent(recvtype, &lb, &recvext);
   XBT_DEBUG("<%d> algorithm basic_alltoallv() called.", rank);
   smpi_datatype_extent(sendtype, &lb, &sendext);
   smpi_datatype_extent(recvtype, &lb, &recvext);
index 8c6cb3e..ed96ac2 100644 (file)
@@ -12,6 +12,7 @@
 #include <xbt/ex.hpp>
 
 #include <simgrid/s4u/host.hpp>
 #include <xbt/ex.hpp>
 
 #include <simgrid/s4u/host.hpp>
+#include <src/smpi/smpi_comm.hpp>
 #include <src/smpi/smpi_group.hpp>
 
 #include "private.h"
 #include <src/smpi/smpi_group.hpp>
 
 #include "private.h"
@@ -27,21 +28,6 @@ int comm_keyval_id = 0;//avoid collisions
 /* Support for cartesian topology was added, but there are 2 other types of topology, graph et dist graph. In order to
  * support them, we have to add a field MPIR_Topo_type, and replace the MPI_Topology field by an union. */
 
 /* Support for cartesian topology was added, but there are 2 other types of topology, graph et dist graph. In order to
  * support them, we have to add a field MPIR_Topo_type, and replace the MPI_Topology field by an union. */
 
-typedef struct s_smpi_mpi_communicator {
-  MPI_Group group;
-  MPIR_Topo_type topoType; 
-  MPI_Topology topo; // to be replaced by an union
-  int refcount;
-  MPI_Comm leaders_comm;//inter-node communicator
-  MPI_Comm intra_comm;//intra-node communicator . For MPI_COMM_WORLD this can't be used, as var is global.
-  //use an intracomm stored in the process data instead
-  int* leaders_map; //who is the leader of each process
-  int is_uniform;
-  int* non_uniform_map; //set if smp nodes have a different number of processes allocated
-  int is_blocked;// are ranks allocated on the same smp node contiguous ?
-  xbt_dict_t attributes;
-} s_smpi_mpi_communicator_t;
-
 static int smpi_compare_rankmap(const void *a, const void *b)
 {
   const int* x = static_cast<const int*>(a);
 static int smpi_compare_rankmap(const void *a, const void *b)
 {
   const int* x = static_cast<const int*>(a);
@@ -62,165 +48,163 @@ static int smpi_compare_rankmap(const void *a, const void *b)
   return 1;
 }
 
   return 1;
 }
 
-MPI_Comm smpi_comm_new(MPI_Group group, MPI_Topology topo)
+namespace simgrid{
+namespace SMPI{
+
+Comm::Comm(MPI_Group group, MPI_Topology topo)
 {
 {
-  MPI_Comm comm;
-
-  comm = xbt_new(s_smpi_mpi_communicator_t, 1);
-  comm->group = group;
-  comm->refcount=1;
-  comm->topoType = MPI_INVALID_TOPO;
-  comm->topo = topo;
-  comm->intra_comm = MPI_COMM_NULL;
-  comm->leaders_comm = MPI_COMM_NULL;
-  comm->is_uniform=1;
-  comm->non_uniform_map = nullptr;
-  comm->leaders_map = nullptr;
-  comm->is_blocked=0;
-  comm->attributes=nullptr;
-  return comm;
+  m_group = group;
+  m_refcount=1;
+  m_topoType = MPI_INVALID_TOPO;
+  m_topo = topo;
+  m_intra_comm = MPI_COMM_NULL;
+  m_leaders_comm = MPI_COMM_NULL;
+  m_is_uniform=1;
+  m_non_uniform_map = nullptr;
+  m_leaders_map = nullptr;
+  m_is_blocked=0;
+  m_attributes=nullptr;
 }
 
 }
 
-void smpi_comm_destroy(MPI_Comm comm)
+void Comm::destroy()
 {
 {
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  smpi_topo_destroy(comm->topo); // there's no use count on topos
-  smpi_comm_unuse(comm);
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->destroy();
+  smpi_topo_destroy(m_topo); // there's no use count on topos
+  this->unuse();
 }
 
 }
 
-int smpi_comm_dup(MPI_Comm comm, MPI_Comm* newcomm){
+int Comm::dup(MPI_Comm* newcomm){
   if(smpi_privatize_global_variables){ //we need to switch as the called function may silently touch global variables
      smpi_switch_data_segment(smpi_process_index());
    }
   if(smpi_privatize_global_variables){ //we need to switch as the called function may silently touch global variables
      smpi_switch_data_segment(smpi_process_index());
    }
-  MPI_Group cp=new simgrid::SMPI::Group(smpi_comm_group(comm));
-  (*newcomm) = smpi_comm_new(cp, smpi_comm_topo(comm));
+  MPI_Group cp = new simgrid::SMPI::Group(this->group());
+  (*newcomm) = new simgrid::SMPI::Comm(cp, this->topo());
   int ret = MPI_SUCCESS;
 
   int ret = MPI_SUCCESS;
 
-  if(comm->attributes !=nullptr){
-    (*newcomm)->attributes   = xbt_dict_new_homogeneous(nullptr);
+  if(m_attributes !=nullptr){
+    (*newcomm)->m_attributes   = xbt_dict_new_homogeneous(nullptr);
     xbt_dict_cursor_t cursor = nullptr;
     char* key;
     int flag;
     void* value_in;
     void* value_out;
     xbt_dict_cursor_t cursor = nullptr;
     char* key;
     int flag;
     void* value_in;
     void* value_out;
-    xbt_dict_foreach (comm->attributes, cursor, key, value_in) {
+    xbt_dict_foreach (m_attributes, cursor, key, value_in) {
       smpi_comm_key_elem elem =
           static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null_ext(smpi_comm_keyvals, key, sizeof(int)));
       if (elem != nullptr && elem->copy_fn != MPI_NULL_COPY_FN) {
       smpi_comm_key_elem elem =
           static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null_ext(smpi_comm_keyvals, key, sizeof(int)));
       if (elem != nullptr && elem->copy_fn != MPI_NULL_COPY_FN) {
-        ret = elem->copy_fn(comm, atoi(key), nullptr, value_in, &value_out, &flag);
+        ret = elem->copy_fn(this, atoi(key), nullptr, value_in, &value_out, &flag);
         if (ret != MPI_SUCCESS) {
         if (ret != MPI_SUCCESS) {
-          smpi_comm_destroy(*newcomm);
+          (*newcomm)->destroy();
           *newcomm = MPI_COMM_NULL;
           xbt_dict_cursor_free(&cursor);
           return ret;
         }
         if (flag)
           *newcomm = MPI_COMM_NULL;
           xbt_dict_cursor_free(&cursor);
           return ret;
         }
         if (flag)
-          xbt_dict_set_ext((*newcomm)->attributes, key, sizeof(int), value_out, nullptr);
+          xbt_dict_set_ext((*newcomm)->m_attributes, key, sizeof(int), value_out, nullptr);
       }
       }
     }
   return ret;
 }
 
       }
       }
     }
   return ret;
 }
 
-MPI_Group smpi_comm_group(MPI_Comm comm)
+MPI_Group Comm::group()
 {
 {
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return comm->group;
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->group();
+  return m_group;
 }
 
 }
 
-MPI_Topology smpi_comm_topo(MPI_Comm comm) {
-  if (comm != MPI_COMM_NULL)
-    return comm->topo;
+MPI_Topology Comm::topo() {
+  if (this != MPI_COMM_NULL)
+    return m_topo;
   return nullptr;
 }
 
   return nullptr;
 }
 
-int smpi_comm_size(MPI_Comm comm)
+int Comm::size()
 {
 {
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return smpi_comm_group(comm)->getsize();
-  return smpi_comm_group(this)->size();
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->size();
+  return m_group->size();
 }
 
 }
 
-int smpi_comm_rank(MPI_Comm comm)
+int Comm::rank()
 {
 {
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return smpi_comm_group(comm)->rank(smpi_process_index());
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->rank();
+  return m_group->rank(smpi_process_index());
 }
 
 }
 
-void smpi_comm_get_name (MPI_Comm comm, char* name, int* len)
+void Comm::get_name (char* name, int* len)
 {
 {
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  if(comm == MPI_COMM_WORLD) {
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->get_name(name, len);
+  if(this == MPI_COMM_WORLD) {
     strncpy(name, "WORLD",5);
     *len = 5;
   } else {
     strncpy(name, "WORLD",5);
     *len = 5;
   } else {
-    *len = snprintf(name, MPI_MAX_NAME_STRING, "%p", comm);
+    *len = snprintf(name, MPI_MAX_NAME_STRING, "%p", this);
   }
 }
 
   }
 }
 
-void smpi_comm_set_leaders_comm(MPI_Comm comm, MPI_Comm leaders){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  comm->leaders_comm=leaders;
+void Comm::set_leaders_comm(MPI_Comm leaders){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->set_leaders_comm(leaders);
+  m_leaders_comm=leaders;
 }
 
 }
 
-void smpi_comm_set_intra_comm(MPI_Comm comm, MPI_Comm leaders){
-  comm->intra_comm=leaders;
+void Comm::set_intra_comm(MPI_Comm leaders){
+  m_intra_comm=leaders;
 }
 
 }
 
-int* smpi_comm_get_non_uniform_map(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return comm->non_uniform_map;
+int* Comm::get_non_uniform_map(){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->get_non_uniform_map();
+  return m_non_uniform_map;
 }
 
 }
 
-int* smpi_comm_get_leaders_map(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return comm->leaders_map;
+int* Comm::get_leaders_map(){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->get_leaders_map();
+  return m_leaders_map;
 }
 
 }
 
-MPI_Comm smpi_comm_get_leaders_comm(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return comm->leaders_comm;
+MPI_Comm Comm::get_leaders_comm(){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->get_leaders_comm();
+  return m_leaders_comm;
 }
 
 }
 
-MPI_Comm smpi_comm_get_intra_comm(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED || comm==MPI_COMM_WORLD) 
+MPI_Comm Comm::get_intra_comm(){
+  if (this == MPI_COMM_UNINITIALIZED || this==MPI_COMM_WORLD) 
     return smpi_process_get_comm_intra();
     return smpi_process_get_comm_intra();
-  else return comm->intra_comm;
+  else return m_intra_comm;
 }
 
 }
 
-int smpi_comm_is_uniform(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return comm->is_uniform;
+int Comm::is_uniform(){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->is_uniform();
+  return m_is_uniform;
 }
 
 }
 
-int smpi_comm_is_blocked(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  return comm->is_blocked;
+int Comm::is_blocked(){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->is_blocked();
+  return m_is_blocked;
 }
 
 }
 
-MPI_Comm smpi_comm_split(MPI_Comm comm, int color, int key)
+MPI_Comm Comm::split(int color, int key)
 {
 {
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->split(color, key);
   int system_tag = 123;
   int* recvbuf;
 
   MPI_Group group_root = nullptr;
   MPI_Group group_out  = nullptr;
   int system_tag = 123;
   int* recvbuf;
 
   MPI_Group group_root = nullptr;
   MPI_Group group_out  = nullptr;
-  MPI_Group group      = smpi_comm_group(comm);
-  int rank             = smpi_comm_rank(comm);
-  int size             = smpi_comm_size(comm);
+  MPI_Group group      = this->group();
+  int rank             = this->rank();
+  int size             = this->size();
   /* Gather all colors and keys on rank 0 */
   int* sendbuf = xbt_new(int, 2);
   sendbuf[0] = color;
   /* Gather all colors and keys on rank 0 */
   int* sendbuf = xbt_new(int, 2);
   sendbuf[0] = color;
@@ -230,7 +214,7 @@ MPI_Comm smpi_comm_split(MPI_Comm comm, int color, int key)
   } else {
     recvbuf = nullptr;
   }
   } else {
     recvbuf = nullptr;
   }
-  smpi_mpi_gather(sendbuf, 2, MPI_INT, recvbuf, 2, MPI_INT, 0, comm);
+  smpi_mpi_gather(sendbuf, 2, MPI_INT, recvbuf, 2, MPI_INT, 0, this);
   xbt_free(sendbuf);
   /* Do the actual job */
   if(rank == 0) {
   xbt_free(sendbuf);
   /* Do the actual job */
   if(rank == 0) {
@@ -266,7 +250,7 @@ MPI_Comm smpi_comm_split(MPI_Comm comm, int color, int key)
         for (int j = 0; j < count; j++) {
           if(rankmap[2 * j] != 0) {
             group_snd[reqs]=new simgrid::SMPI::Group(group_out);
         for (int j = 0; j < count; j++) {
           if(rankmap[2 * j] != 0) {
             group_snd[reqs]=new simgrid::SMPI::Group(group_out);
-            requests[reqs] = smpi_mpi_isend(&(group_snd[reqs]), 1, MPI_PTR, rankmap[2 * j], system_tag, comm);
+            requests[reqs] = smpi_mpi_isend(&(group_snd[reqs]), 1, MPI_PTR, rankmap[2 * j], system_tag, this);
             reqs++;
           }
         }
             reqs++;
           }
         }
@@ -283,55 +267,55 @@ MPI_Comm smpi_comm_split(MPI_Comm comm, int color, int key)
     group_out = group_root; /* exit with root's group */
   } else {
     if(color != MPI_UNDEFINED) {
     group_out = group_root; /* exit with root's group */
   } else {
     if(color != MPI_UNDEFINED) {
-      smpi_mpi_recv(&group_out, 1, MPI_PTR, 0, system_tag, comm, MPI_STATUS_IGNORE);
+      smpi_mpi_recv(&group_out, 1, MPI_PTR, 0, system_tag, this, MPI_STATUS_IGNORE);
     } /* otherwise, exit with group_out == nullptr */
   }
     } /* otherwise, exit with group_out == nullptr */
   }
-  return group_out!=nullptr ? smpi_comm_new(group_out, nullptr) : MPI_COMM_NULL;
+  return group_out!=nullptr ? new simgrid::SMPI::Comm(group_out, nullptr) : MPI_COMM_NULL;
 }
 
 }
 
-void smpi_comm_use(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  comm->group->use();
-  comm->refcount++;
+void Comm::use(){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->use();
+  m_group->use();
+  m_refcount++;
 }
 
 }
 
-void smpi_comm_cleanup_attributes(MPI_Comm comm){
-  if(comm->attributes !=nullptr){
+void Comm::cleanup_attributes(){
+  if(m_attributes !=nullptr){
     xbt_dict_cursor_t cursor = nullptr;
     char* key;
     void* value;
     int flag;
     xbt_dict_cursor_t cursor = nullptr;
     char* key;
     void* value;
     int flag;
-    xbt_dict_foreach (comm->attributes, cursor, key, value) {
+    xbt_dict_foreach (m_attributes, cursor, key, value) {
       smpi_comm_key_elem elem = static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null(smpi_comm_keyvals, key));
       if (elem != nullptr && elem->delete_fn != nullptr)
       smpi_comm_key_elem elem = static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null(smpi_comm_keyvals, key));
       if (elem != nullptr && elem->delete_fn != nullptr)
-        elem->delete_fn(comm, atoi(key), value, &flag);
+        elem->delete_fn(this, atoi(key), value, &flag);
     }
     }
-    xbt_dict_free(&comm->attributes);
+    xbt_dict_free(&m_attributes);
   }
 }
 
   }
 }
 
-void smpi_comm_cleanup_smp(MPI_Comm comm){
-  if (comm->intra_comm != MPI_COMM_NULL)
-    smpi_comm_unuse(comm->intra_comm);
-  if (comm->leaders_comm != MPI_COMM_NULL)
-    smpi_comm_unuse(comm->leaders_comm);
-  if (comm->non_uniform_map != nullptr)
-    xbt_free(comm->non_uniform_map);
-  if (comm->leaders_map != nullptr)
-    xbt_free(comm->leaders_map);
+void Comm::cleanup_smp(){
+  if (m_intra_comm != MPI_COMM_NULL)
+    m_intra_comm->unuse();
+  if (m_leaders_comm != MPI_COMM_NULL)
+    m_leaders_comm->unuse();
+  if (m_non_uniform_map != nullptr)
+    xbt_free(m_non_uniform_map);
+  if (m_leaders_map != nullptr)
+    xbt_free(m_leaders_map);
 }
 
 }
 
-void smpi_comm_unuse(MPI_Comm comm){
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
-  comm->refcount--;
-  comm->group->unuse();
+void Comm::unuse(){
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->unuse();
+  m_refcount--;
+  m_group->unuse();
 
 
-  if(comm->refcount==0){
-    smpi_comm_cleanup_smp(comm);
-    smpi_comm_cleanup_attributes(comm);
-    xbt_free(comm);
+  if(m_refcount==0){
+    this->cleanup_smp();
+    this->cleanup_attributes();
+    delete this;
   }
 }
 
   }
 }
 
@@ -343,13 +327,13 @@ static int compare_ints (const void *a, const void *b)
   return static_cast<int>(*da > *db) - static_cast<int>(*da < *db);
 }
 
   return static_cast<int>(*da > *db) - static_cast<int>(*da < *db);
 }
 
-void smpi_comm_init_smp(MPI_Comm comm){
+void Comm::init_smp(){
   int leader = -1;
 
   int leader = -1;
 
-  if (comm == MPI_COMM_UNINITIALIZED)
-    comm = smpi_process_comm_world();
+  if (this == MPI_COMM_UNINITIALIZED)
+    return smpi_process_comm_world()->init_smp();
 
 
-  int comm_size =smpi_comm_size(comm);
+  int comm_size = this->size();
   
   // If we are in replay - perform an ugly hack  
   // tell SimGrid we are not in replay for a while, because we need the buffers to be copied for the following calls
   
   // If we are in replay - perform an ugly hack  
   // tell SimGrid we are not in replay for a while, because we need the buffers to be copied for the following calls
@@ -372,7 +356,7 @@ void smpi_comm_init_smp(MPI_Comm comm){
   xbt_swag_foreach(process, process_list) {
     int index = process->pid -1;
 
   xbt_swag_foreach(process, process_list) {
     int index = process->pid -1;
 
-    if(smpi_comm_group(comm)->rank(index)!=MPI_UNDEFINED){
+    if(this->group()->rank(index)!=MPI_UNDEFINED){
         intra_comm_size++;
       //the process is in the comm
       if(index < min_index)
         intra_comm_size++;
       //the process is in the comm
       if(index < min_index)
@@ -386,13 +370,13 @@ void smpi_comm_init_smp(MPI_Comm comm){
   process = nullptr;
   xbt_swag_foreach(process, process_list) {
     int index = process->pid -1;
   process = nullptr;
   xbt_swag_foreach(process, process_list) {
     int index = process->pid -1;
-    if(smpi_comm_group(comm)->rank(index)!=MPI_UNDEFINED){
+    if(this->group()->rank(index)!=MPI_UNDEFINED){
       group_intra->set_mapping(index, i);
       i++;
     }
   }
 
       group_intra->set_mapping(index, i);
       i++;
     }
   }
 
-  MPI_Comm comm_intra = smpi_comm_new(group_intra, nullptr);
+  MPI_Comm comm_intra = new simgrid::SMPI::Comm(group_intra, nullptr);
   leader=min_index;
 
   int * leaders_map= static_cast<int*>(xbt_malloc0(sizeof(int)*comm_size));
   leader=min_index;
 
   int * leaders_map= static_cast<int*>(xbt_malloc0(sizeof(int)*comm_size));
@@ -401,14 +385,14 @@ void smpi_comm_init_smp(MPI_Comm comm){
       leader_list[i]=-1;
   }
 
       leader_list[i]=-1;
   }
 
-  smpi_coll_tuned_allgather_mpich(&leader, 1, MPI_INT , leaders_map, 1, MPI_INT, comm);
+  smpi_coll_tuned_allgather_mpich(&leader, 1, MPI_INT , leaders_map, 1, MPI_INT, this);
 
   if(smpi_privatize_global_variables){ //we need to switch as the called function may silently touch global variables
      smpi_switch_data_segment(smpi_process_index());
    }
 
 
   if(smpi_privatize_global_variables){ //we need to switch as the called function may silently touch global variables
      smpi_switch_data_segment(smpi_process_index());
    }
 
-  if(comm->leaders_map==nullptr){
-    comm->leaders_map= leaders_map;
+  if(m_leaders_map==nullptr){
+    m_leaders_map= leaders_map;
   }else{
     xbt_free(leaders_map);
   }
   }else{
     xbt_free(leaders_map);
   }
@@ -417,12 +401,12 @@ void smpi_comm_init_smp(MPI_Comm comm){
   for(i=0; i<comm_size; i++){
       int already_done=0;
       for(j=0;j<leader_group_size; j++){
   for(i=0; i<comm_size; i++){
       int already_done=0;
       for(j=0;j<leader_group_size; j++){
-        if(comm->leaders_map[i]==leader_list[j]){
+        if(m_leaders_map[i]==leader_list[j]){
             already_done=1;
         }
       }
       if(already_done==0){
             already_done=1;
         }
       }
       if(already_done==0){
-        leader_list[leader_group_size]=comm->leaders_map[i];
+        leader_list[leader_group_size]=m_leaders_map[i];
         leader_group_size++;
       }
   }
         leader_group_size++;
       }
   }
@@ -431,24 +415,24 @@ void smpi_comm_init_smp(MPI_Comm comm){
   MPI_Group leaders_group = new simgrid::SMPI::Group(leader_group_size);
 
   MPI_Comm leader_comm = MPI_COMM_NULL;
   MPI_Group leaders_group = new simgrid::SMPI::Group(leader_group_size);
 
   MPI_Comm leader_comm = MPI_COMM_NULL;
-  if(MPI_COMM_WORLD!=MPI_COMM_UNINITIALIZED && comm!=MPI_COMM_WORLD){
+  if(MPI_COMM_WORLD!=MPI_COMM_UNINITIALIZED && this!=MPI_COMM_WORLD){
     //create leader_communicator
     for (i=0; i< leader_group_size;i++)
       leaders_group->set_mapping(leader_list[i], i);
     //create leader_communicator
     for (i=0; i< leader_group_size;i++)
       leaders_group->set_mapping(leader_list[i], i);
-    leader_comm = smpi_comm_new(leaders_group, nullptr);
-    smpi_comm_set_leaders_comm(comm, leader_comm);
-    smpi_comm_set_intra_comm(comm, comm_intra);
+    leader_comm = new simgrid::SMPI::Comm(leaders_group, nullptr);
+    this->set_leaders_comm(leader_comm);
+    this->set_intra_comm(comm_intra);
 
    //create intracommunicator
   }else{
     for (i=0; i< leader_group_size;i++)
       leaders_group->set_mapping(leader_list[i], i);
 
 
    //create intracommunicator
   }else{
     for (i=0; i< leader_group_size;i++)
       leaders_group->set_mapping(leader_list[i], i);
 
-    if(smpi_comm_get_leaders_comm(comm)==MPI_COMM_NULL){
-      leader_comm = smpi_comm_new(leaders_group, nullptr);
-      smpi_comm_set_leaders_comm(comm, leader_comm);
+    if(this->get_leaders_comm()==MPI_COMM_NULL){
+      leader_comm = new simgrid::SMPI::Comm(leaders_group, nullptr);
+      this->set_leaders_comm(leader_comm);
     }else{
     }else{
-      leader_comm=smpi_comm_get_leaders_comm(comm);
+      leader_comm=this->get_leaders_comm();
       leaders_group->unuse();
     }
     smpi_process_set_comm_intra(comm_intra);
       leaders_group->unuse();
     }
     smpi_process_set_comm_intra(comm_intra);
@@ -457,8 +441,8 @@ void smpi_comm_init_smp(MPI_Comm comm){
   int is_uniform = 1;
 
   // Are the nodes uniform ? = same number of process/node
   int is_uniform = 1;
 
   // Are the nodes uniform ? = same number of process/node
-  int my_local_size=smpi_comm_size(comm_intra);
-  if(smpi_comm_rank(comm_intra)==0) {
+  int my_local_size=comm_intra->size();
+  if(comm_intra->rank()==0) {
     int* non_uniform_map = xbt_new0(int,leader_group_size);
     smpi_coll_tuned_allgather_mpich(&my_local_size, 1, MPI_INT,
         non_uniform_map, 1, MPI_INT, leader_comm);
     int* non_uniform_map = xbt_new0(int,leader_group_size);
     smpi_coll_tuned_allgather_mpich(&my_local_size, 1, MPI_INT,
         non_uniform_map, 1, MPI_INT, leader_comm);
@@ -468,23 +452,23 @@ void smpi_comm_init_smp(MPI_Comm comm){
         break;
       }
     }
         break;
       }
     }
-    if(is_uniform==0 && smpi_comm_is_uniform(comm)!=0){
-        comm->non_uniform_map= non_uniform_map;
+    if(is_uniform==0 && this->is_uniform()!=0){
+        m_non_uniform_map= non_uniform_map;
     }else{
         xbt_free(non_uniform_map);
     }
     }else{
         xbt_free(non_uniform_map);
     }
-    comm->is_uniform=is_uniform;
+    m_is_uniform=is_uniform;
   }
   }
-  smpi_coll_tuned_bcast_mpich(&(comm->is_uniform),1, MPI_INT, 0, comm_intra );
+  smpi_coll_tuned_bcast_mpich(&(m_is_uniform),1, MPI_INT, 0, comm_intra );
 
   if(smpi_privatize_global_variables){ //we need to switch as the called function may silently touch global variables
      smpi_switch_data_segment(smpi_process_index());
    }
   // Are the ranks blocked ? = allocated contiguously on the SMP nodes
   int is_blocked=1;
 
   if(smpi_privatize_global_variables){ //we need to switch as the called function may silently touch global variables
      smpi_switch_data_segment(smpi_process_index());
    }
   // Are the ranks blocked ? = allocated contiguously on the SMP nodes
   int is_blocked=1;
-  int prev=smpi_comm_group(comm)->rank(smpi_comm_group(comm_intra)->index(0));
+  int prev=this->group()->rank(comm_intra->group()->index(0));
     for (i=1; i<my_local_size; i++){
     for (i=1; i<my_local_size; i++){
-      int that=smpi_comm_group(comm)->rank(smpi_comm_group(comm_intra)->index(i));
+      int that=this->group()->rank(comm_intra->group()->index(i));
       if(that!=prev+1){
         is_blocked=0;
         break;
       if(that!=prev+1){
         is_blocked=0;
         break;
@@ -493,14 +477,14 @@ void smpi_comm_init_smp(MPI_Comm comm){
   }
 
   int global_blocked;
   }
 
   int global_blocked;
-  smpi_mpi_allreduce(&is_blocked, &(global_blocked), 1, MPI_INT, MPI_LAND, comm);
+  smpi_mpi_allreduce(&is_blocked, &(global_blocked), 1, MPI_INT, MPI_LAND, this);
 
 
-  if(MPI_COMM_WORLD==MPI_COMM_UNINITIALIZED || comm==MPI_COMM_WORLD){
-    if(smpi_comm_rank(comm)==0){
-        comm->is_blocked=global_blocked;
+  if(MPI_COMM_WORLD==MPI_COMM_UNINITIALIZED || this==MPI_COMM_WORLD){
+    if(this->rank()==0){
+        m_is_blocked=global_blocked;
     }
   }else{
     }
   }else{
-    comm->is_blocked=global_blocked;
+    m_is_blocked=global_blocked;
   }
   xbt_free(leader_list);
   
   }
   xbt_free(leader_list);
   
@@ -508,7 +492,7 @@ void smpi_comm_init_smp(MPI_Comm comm){
     smpi_process_set_replaying(true); 
 }
 
     smpi_process_set_replaying(true); 
 }
 
-int smpi_comm_attr_delete(MPI_Comm comm, int keyval){
+int Comm::attr_delete(int keyval){
   smpi_comm_key_elem elem =
      static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null_ext(smpi_comm_keyvals, reinterpret_cast<const char*>(&keyval), sizeof(int)));
   if(elem==nullptr)
   smpi_comm_key_elem elem =
      static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null_ext(smpi_comm_keyvals, reinterpret_cast<const char*>(&keyval), sizeof(int)));
   if(elem==nullptr)
@@ -516,31 +500,31 @@ int smpi_comm_attr_delete(MPI_Comm comm, int keyval){
   if(elem->delete_fn!=MPI_NULL_DELETE_FN){
     void* value = nullptr;
     int flag;
   if(elem->delete_fn!=MPI_NULL_DELETE_FN){
     void* value = nullptr;
     int flag;
-    if(smpi_comm_attr_get(comm, keyval, &value, &flag)==MPI_SUCCESS){
-      int ret = elem->delete_fn(comm, keyval, value, &flag);
+    if(this->attr_get(keyval, &value, &flag)==MPI_SUCCESS){
+      int ret = elem->delete_fn(this, keyval, value, &flag);
       if(ret!=MPI_SUCCESS) 
         return ret;
     }
   }
       if(ret!=MPI_SUCCESS) 
         return ret;
     }
   }
-  if(comm->attributes==nullptr)
+  if(m_attributes==nullptr)
     return MPI_ERR_ARG;
 
     return MPI_ERR_ARG;
 
-  xbt_dict_remove_ext(comm->attributes, reinterpret_cast<const char*>(&keyval), sizeof(int));
+  xbt_dict_remove_ext(m_attributes, reinterpret_cast<const char*>(&keyval), sizeof(int));
   return MPI_SUCCESS;
 }
 
   return MPI_SUCCESS;
 }
 
-int smpi_comm_attr_get(MPI_Comm comm, int keyval, void* attr_value, int* flag){
+int Comm::attr_get(int keyval, void* attr_value, int* flag){
   smpi_comm_key_elem elem =
     static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null_ext(smpi_comm_keyvals, reinterpret_cast<const char*>(&keyval), sizeof(int)));
   if(elem==nullptr)
     return MPI_ERR_ARG;
   smpi_comm_key_elem elem =
     static_cast<smpi_comm_key_elem>(xbt_dict_get_or_null_ext(smpi_comm_keyvals, reinterpret_cast<const char*>(&keyval), sizeof(int)));
   if(elem==nullptr)
     return MPI_ERR_ARG;
-  if(comm->attributes==nullptr){
+  if(m_attributes==nullptr){
     *flag=0;
     return MPI_SUCCESS;
   }
   try {
     *static_cast<void**>(attr_value) =
     *flag=0;
     return MPI_SUCCESS;
   }
   try {
     *static_cast<void**>(attr_value) =
-        xbt_dict_get_ext(comm->attributes, reinterpret_cast<const char*>(&keyval), sizeof(int));
+        xbt_dict_get_ext(m_attributes, reinterpret_cast<const char*>(&keyval), sizeof(int));
     *flag=1;
   }
   catch (xbt_ex& ex) {
     *flag=1;
   }
   catch (xbt_ex& ex) {
@@ -549,7 +533,7 @@ int smpi_comm_attr_get(MPI_Comm comm, int keyval, void* attr_value, int* flag){
   return MPI_SUCCESS;
 }
 
   return MPI_SUCCESS;
 }
 
-int smpi_comm_attr_put(MPI_Comm comm, int keyval, void* attr_value){
+int Comm::attr_put(int keyval, void* attr_value){
   if(smpi_comm_keyvals==nullptr)
     smpi_comm_keyvals = xbt_dict_new_homogeneous(nullptr);
   smpi_comm_key_elem elem =
   if(smpi_comm_keyvals==nullptr)
     smpi_comm_keyvals = xbt_dict_new_homogeneous(nullptr);
   smpi_comm_key_elem elem =
@@ -558,19 +542,22 @@ int smpi_comm_attr_put(MPI_Comm comm, int keyval, void* attr_value){
     return MPI_ERR_ARG;
   int flag;
   void* value = nullptr;
     return MPI_ERR_ARG;
   int flag;
   void* value = nullptr;
-  smpi_comm_attr_get(comm, keyval, &value, &flag);
+  this->attr_get(keyval, &value, &flag);
   if(flag!=0 && elem->delete_fn!=MPI_NULL_DELETE_FN){
   if(flag!=0 && elem->delete_fn!=MPI_NULL_DELETE_FN){
-    int ret = elem->delete_fn(comm, keyval, value, &flag);
+    int ret = elem->delete_fn(this, keyval, value, &flag);
     if(ret!=MPI_SUCCESS) 
       return ret;
   }
     if(ret!=MPI_SUCCESS) 
       return ret;
   }
-  if(comm->attributes==nullptr)
-    comm->attributes = xbt_dict_new_homogeneous(nullptr);
+  if(m_attributes==nullptr)
+    m_attributes = xbt_dict_new_homogeneous(nullptr);
 
 
-  xbt_dict_set_ext(comm->attributes,  reinterpret_cast<const char*>(&keyval), sizeof(int), attr_value, nullptr);
+  xbt_dict_set_ext(m_attributes,  reinterpret_cast<const char*>(&keyval), sizeof(int), attr_value, nullptr);
   return MPI_SUCCESS;
 }
 
   return MPI_SUCCESS;
 }
 
+}
+}
+
 int smpi_comm_keyval_create(MPI_Comm_copy_attr_function* copy_fn, MPI_Comm_delete_attr_function* delete_fn, int* keyval,
                             void* extra_state){
   if(smpi_comm_keyvals==nullptr)
 int smpi_comm_keyval_create(MPI_Comm_copy_attr_function* copy_fn, MPI_Comm_delete_attr_function* delete_fn, int* keyval,
                             void* extra_state){
   if(smpi_comm_keyvals==nullptr)
diff --git a/src/smpi/smpi_comm.hpp b/src/smpi/smpi_comm.hpp
new file mode 100644 (file)
index 0000000..1b71731
--- /dev/null
@@ -0,0 +1,66 @@
+/* Copyright (c) 2010-2015. The SimGrid Team.
+ * All rights reserved.                                                     */
+
+/* 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. */
+
+#ifndef SMPI_COMM_HPP_INCLUDED
+#define SMPI_COMM_HPP_INCLUDED
+
+#include "private.h"
+
+namespace simgrid{
+namespace SMPI{
+
+class Comm {
+
+  private:
+    MPI_Group m_group;
+    MPIR_Topo_type m_topoType; 
+    MPI_Topology m_topo; // to be replaced by an union
+    int m_refcount;
+    MPI_Comm m_leaders_comm;//inter-node communicator
+    MPI_Comm m_intra_comm;//intra-node communicator . For MPI_COMM_WORLD this can't be used, as var is global.
+    //use an intracomm stored in the process data instead
+    int* m_leaders_map; //who is the leader of each process
+    int m_is_uniform;
+    int* m_non_uniform_map; //set if smp nodes have a different number of processes allocated
+    int m_is_blocked;// are ranks allocated on the same smp node contiguous ?
+    xbt_dict_t m_attributes;
+
+  public:
+
+    Comm(MPI_Group group, MPI_Topology topo);
+
+    void destroy();
+    int dup(MPI_Comm* newcomm);
+    MPI_Group group();
+    MPI_Topology topo();
+    int size();
+    int rank();
+    void get_name (char* name, int* len);
+    void set_leaders_comm(MPI_Comm leaders);
+    void set_intra_comm(MPI_Comm leaders);
+    int* get_non_uniform_map();
+    int* get_leaders_map();
+    MPI_Comm get_leaders_comm();
+    MPI_Comm get_intra_comm();
+    int is_uniform();
+    int is_blocked();
+    MPI_Comm split(int color, int key);
+    void use();
+    void cleanup_attributes();
+    void cleanup_smp();
+    void unuse();
+    void init_smp();
+    int attr_delete(int keyval);
+    int attr_get(int keyval, void* attr_value, int* flag);
+    int attr_put(int keyval, void* attr_value);
+
+};
+
+}
+}
+
+
+#endif
index fc75d75..67a0cea 100644 (file)
@@ -71,11 +71,11 @@ void smpi_deployment_register_process(const char* instance_id, int rank, int ind
 
   if(instance->comm_world == MPI_COMM_NULL){
     MPI_Group group = new simgrid::SMPI::Group(instance->size);
 
   if(instance->comm_world == MPI_COMM_NULL){
     MPI_Group group = new simgrid::SMPI::Group(instance->size);
-    instance->comm_world = smpi_comm_new(group, nullptr);
+    instance->comm_world = new simgrid::SMPI::Comm(group, nullptr);
   }
   instance->present_processes++;
   index_to_process_data[index]=instance->index+rank;
   }
   instance->present_processes++;
   index_to_process_data[index]=instance->index+rank;
-  smpi_comm_group(instance->comm_world)->set_mapping(index, rank);
+  instance->comm_world->group()->set_mapping(index, rank);
   *bar = instance->finalization_barrier;
   *comm = &instance->comm_world;
 }
   *bar = instance->finalization_barrier;
   *comm = &instance->comm_world;
 }
@@ -86,7 +86,7 @@ void smpi_deployment_cleanup_instances(){
   char *name = nullptr;
   xbt_dict_foreach(smpi_instances, cursor, name, instance) {
     if(instance->comm_world!=MPI_COMM_NULL)
   char *name = nullptr;
   xbt_dict_foreach(smpi_instances, cursor, name, instance) {
     if(instance->comm_world!=MPI_COMM_NULL)
-      while (smpi_comm_group(instance->comm_world)->unuse() > 0);
+      while (instance->comm_world->group()->unuse() > 0);
     xbt_free(instance->comm_world);
     MSG_barrier_destroy(instance->finalization_barrier);
   }
     xbt_free(instance->comm_world);
     MSG_barrier_destroy(instance->finalization_barrier);
   }
index 2ecc4f3..834467a 100644 (file)
@@ -354,7 +354,7 @@ MPI_Comm smpi_process_comm_self()
   smpi_process_data_t data = smpi_process_data();
   if(data->comm_self==MPI_COMM_NULL){
     MPI_Group group = new simgrid::SMPI::Group(1);
   smpi_process_data_t data = smpi_process_data();
   if(data->comm_self==MPI_COMM_NULL){
     MPI_Group group = new simgrid::SMPI::Group(1);
-    data->comm_self = smpi_comm_new(group, nullptr);
+    data->comm_self = new simgrid::SMPI::Comm(group, nullptr);
     group->set_mapping(smpi_process_index(), 0);
   }
 
     group->set_mapping(smpi_process_index(), 0);
   }
 
@@ -606,7 +606,7 @@ void smpi_global_init()
   //if not, we let MPI_COMM_NULL, and the comm world will be private to each mpi instance
   if(smpirun){
     group = new simgrid::SMPI::Group(process_count);
   //if not, we let MPI_COMM_NULL, and the comm world will be private to each mpi instance
   if(smpirun){
     group = new simgrid::SMPI::Group(process_count);
-    MPI_COMM_WORLD = smpi_comm_new(group, nullptr);
+    MPI_COMM_WORLD = new simgrid::SMPI::Comm(group, nullptr);
     MPI_Attr_put(MPI_COMM_WORLD, MPI_UNIVERSE_SIZE, reinterpret_cast<void *>(process_count));
     msg_bar_t bar = MSG_barrier_init(process_count);
 
     MPI_Attr_put(MPI_COMM_WORLD, MPI_UNIVERSE_SIZE, reinterpret_cast<void *>(process_count));
     msg_bar_t bar = MSG_barrier_init(process_count);
 
@@ -623,17 +623,17 @@ void smpi_global_destroy()
 
   smpi_bench_destroy();
   if (MPI_COMM_WORLD != MPI_COMM_UNINITIALIZED){
 
   smpi_bench_destroy();
   if (MPI_COMM_WORLD != MPI_COMM_UNINITIALIZED){
-      while (smpi_comm_group(MPI_COMM_WORLD)->unuse() > 0);
+      while (MPI_COMM_WORLD->group()->unuse() > 0);
       MSG_barrier_destroy(process_data[0]->finalization_barrier);
   }else{
       smpi_deployment_cleanup_instances();
   }
   for (int i = 0; i < count; i++) {
     if(process_data[i]->comm_self!=MPI_COMM_NULL){
       MSG_barrier_destroy(process_data[0]->finalization_barrier);
   }else{
       smpi_deployment_cleanup_instances();
   }
   for (int i = 0; i < count; i++) {
     if(process_data[i]->comm_self!=MPI_COMM_NULL){
-      smpi_comm_destroy(process_data[i]->comm_self);
+      process_data[i]->comm_self->destroy();
     }
     if(process_data[i]->comm_intra!=MPI_COMM_NULL){
     }
     if(process_data[i]->comm_intra!=MPI_COMM_NULL){
-      smpi_comm_destroy(process_data[i]->comm_intra);
+      process_data[i]->comm_intra->destroy();
     }
     xbt_os_timer_free(process_data[i]->timer);
     xbt_mutex_destroy(process_data[i]->mailboxes_mutex);
     }
     xbt_os_timer_free(process_data[i]->timer);
     xbt_mutex_destroy(process_data[i]->mailboxes_mutex);
@@ -643,8 +643,8 @@ void smpi_global_destroy()
   process_data = nullptr;
 
   if (MPI_COMM_WORLD != MPI_COMM_UNINITIALIZED){
   process_data = nullptr;
 
   if (MPI_COMM_WORLD != MPI_COMM_UNINITIALIZED){
-    smpi_comm_cleanup_smp(MPI_COMM_WORLD);
-    smpi_comm_cleanup_attributes(MPI_COMM_WORLD);
+    MPI_COMM_WORLD->cleanup_smp();
+    MPI_COMM_WORLD->cleanup_attributes();
     if(smpi_coll_cleanup_callback!=nullptr)
       smpi_coll_cleanup_callback();
     xbt_free(MPI_COMM_WORLD);
     if(smpi_coll_cleanup_callback!=nullptr)
       smpi_coll_cleanup_callback();
     xbt_free(MPI_COMM_WORLD);
index 6fc5084..e8ccfa3 100644 (file)
@@ -69,7 +69,7 @@ Group::~Group()
 
 void Group::destroy()
 {
 
 void Group::destroy()
 {
-  if(this != smpi_comm_group(MPI_COMM_WORLD)
+  if(this != MPI_COMM_WORLD->group()
           && this != MPI_GROUP_NULL
           && this != MPI_GROUP_EMPTY)
   this->unuse();
           && this != MPI_GROUP_NULL
           && this != MPI_GROUP_EMPTY)
   this->unuse();
@@ -169,9 +169,9 @@ int Group::incl(int n, int* ranks, MPI_Group* newgroup)
     *newgroup = MPI_GROUP_EMPTY;
   } else if (n == m_size) {
     *newgroup = this;
     *newgroup = MPI_GROUP_EMPTY;
   } else if (n == m_size) {
     *newgroup = this;
-    if(this!= smpi_comm_group(MPI_COMM_WORLD)
+    if(this!= MPI_COMM_WORLD->group()
               && this != MPI_GROUP_NULL
               && this != MPI_GROUP_NULL
-              && this != smpi_comm_group(MPI_COMM_SELF)
+              && this != MPI_COMM_SELF->group()
               && this != MPI_GROUP_EMPTY)
     this->use();
   } else {
               && this != MPI_GROUP_EMPTY)
     this->use();
   } else {
index 15bb739..6d660d0 100644 (file)
@@ -385,8 +385,8 @@ int PMPI_Group_excl(MPI_Group group, int n, int *ranks, MPI_Group * newgroup)
   } else {
     if (n == 0) {
       *newgroup = group;
   } else {
     if (n == 0) {
       *newgroup = group;
-      if (group != smpi_comm_group(MPI_COMM_WORLD)
-                && group != smpi_comm_group(MPI_COMM_SELF) && group != MPI_GROUP_EMPTY)
+      if (group != MPI_COMM_WORLD->group()
+                && group != MPI_COMM_SELF->group() && group != MPI_GROUP_EMPTY)
       group->use();
       return MPI_SUCCESS;
     } else if (n == group->size()) {
       group->use();
       return MPI_SUCCESS;
     } else if (n == group->size()) {
@@ -423,7 +423,7 @@ int PMPI_Group_range_excl(MPI_Group group, int n, int ranges[][3], MPI_Group * n
   } else {
     if (n == 0) {
       *newgroup = group;
   } else {
     if (n == 0) {
       *newgroup = group;
-      if (group != smpi_comm_group(MPI_COMM_WORLD) && group != smpi_comm_group(MPI_COMM_SELF) &&
+      if (group != MPI_COMM_WORLD->group() && group != MPI_COMM_SELF->group() &&
           group != MPI_GROUP_EMPTY)
         group->use();
       return MPI_SUCCESS;
           group != MPI_GROUP_EMPTY)
         group->use();
       return MPI_SUCCESS;
@@ -440,7 +440,7 @@ int PMPI_Comm_rank(MPI_Comm comm, int *rank)
   } else if (rank == nullptr) {
     return MPI_ERR_ARG;
   } else {
   } else if (rank == nullptr) {
     return MPI_ERR_ARG;
   } else {
-    *rank = smpi_comm_rank(comm);
+    *rank = comm->rank();
     return MPI_SUCCESS;
   }
 }
     return MPI_SUCCESS;
   }
 }
@@ -452,7 +452,7 @@ int PMPI_Comm_size(MPI_Comm comm, int *size)
   } else if (size == nullptr) {
     return MPI_ERR_ARG;
   } else {
   } else if (size == nullptr) {
     return MPI_ERR_ARG;
   } else {
-    *size = smpi_comm_size(comm);
+    *size = comm->size();
     return MPI_SUCCESS;
   }
 }
     return MPI_SUCCESS;
   }
 }
@@ -464,7 +464,7 @@ int PMPI_Comm_get_name (MPI_Comm comm, char* name, int* len)
   } else if (name == nullptr || len == nullptr)  {
     return MPI_ERR_ARG;
   } else {
   } else if (name == nullptr || len == nullptr)  {
     return MPI_ERR_ARG;
   } else {
-    smpi_comm_get_name(comm, name, len);
+    comm->get_name(name, len);
     return MPI_SUCCESS;
   }
 }
     return MPI_SUCCESS;
   }
 }
@@ -476,8 +476,8 @@ int PMPI_Comm_group(MPI_Comm comm, MPI_Group * group)
   } else if (group == nullptr) {
     return MPI_ERR_ARG;
   } else {
   } else if (group == nullptr) {
     return MPI_ERR_ARG;
   } else {
-    *group = smpi_comm_group(comm);
-    if (*group != smpi_comm_group(MPI_COMM_WORLD) && *group != MPI_GROUP_NULL && *group != MPI_GROUP_EMPTY)
+    *group = comm->group();
+    if (*group != MPI_COMM_WORLD->group() && *group != MPI_GROUP_NULL && *group != MPI_GROUP_EMPTY)
       (*group)->use();
     return MPI_SUCCESS;
   }
       (*group)->use();
     return MPI_SUCCESS;
   }
@@ -493,7 +493,7 @@ int PMPI_Comm_compare(MPI_Comm comm1, MPI_Comm comm2, int *result)
     if (comm1 == comm2) {       /* Same communicators means same groups */
       *result = MPI_IDENT;
     } else {
     if (comm1 == comm2) {       /* Same communicators means same groups */
       *result = MPI_IDENT;
     } else {
-      *result = smpi_comm_group(comm1)->compare(smpi_comm_group(comm2));
+      *result = comm1->group()->compare(comm2->group());
       if (*result == MPI_IDENT) {
         *result = MPI_CONGRUENT;
       }
       if (*result == MPI_IDENT) {
         *result = MPI_CONGRUENT;
       }
@@ -509,7 +509,7 @@ int PMPI_Comm_dup(MPI_Comm comm, MPI_Comm * newcomm)
   } else if (newcomm == nullptr) {
     return MPI_ERR_ARG;
   } else {
   } else if (newcomm == nullptr) {
     return MPI_ERR_ARG;
   } else {
-    return smpi_comm_dup(comm, newcomm);
+    return comm->dup(newcomm);
   }
 }
 
   }
 }
 
@@ -526,7 +526,7 @@ int PMPI_Comm_create(MPI_Comm comm, MPI_Group group, MPI_Comm * newcomm)
     return MPI_SUCCESS;
   }else{
     group->use();
     return MPI_SUCCESS;
   }else{
     group->use();
-    *newcomm = smpi_comm_new(group, nullptr);
+    *newcomm = new simgrid::SMPI::Comm(group, nullptr);
     return MPI_SUCCESS;
   }
 }
     return MPI_SUCCESS;
   }
 }
@@ -538,7 +538,7 @@ int PMPI_Comm_free(MPI_Comm * comm)
   } else if (*comm == MPI_COMM_NULL) {
     return MPI_ERR_COMM;
   } else {
   } else if (*comm == MPI_COMM_NULL) {
     return MPI_ERR_COMM;
   } else {
-    smpi_comm_destroy(*comm);
+    (*comm)->destroy();
     *comm = MPI_COMM_NULL;
     return MPI_SUCCESS;
   }
     *comm = MPI_COMM_NULL;
     return MPI_SUCCESS;
   }
@@ -552,7 +552,7 @@ int PMPI_Comm_disconnect(MPI_Comm * comm)
   } else if (*comm == MPI_COMM_NULL) {
     return MPI_ERR_COMM;
   } else {
   } else if (*comm == MPI_COMM_NULL) {
     return MPI_ERR_COMM;
   } else {
-    smpi_comm_destroy(*comm);
+    (*comm)->destroy();
     *comm = MPI_COMM_NULL;
     return MPI_SUCCESS;
   }
     *comm = MPI_COMM_NULL;
     return MPI_SUCCESS;
   }
@@ -568,7 +568,7 @@ int PMPI_Comm_split(MPI_Comm comm, int color, int key, MPI_Comm* comm_out)
   } else if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
   } else {
   } else if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
   } else {
-    *comm_out = smpi_comm_split(comm, color, key);
+    *comm_out = comm->split(color, key);
     retval = MPI_SUCCESS;
   }
   smpi_bench_begin();
     retval = MPI_SUCCESS;
   }
   smpi_bench_begin();
@@ -726,7 +726,7 @@ int PMPI_Irecv(void *buf, int count, MPI_Datatype datatype, int src, int tag, MP
   } else if (src == MPI_PROC_NULL) {
     *request = MPI_REQUEST_NULL;
     retval = MPI_SUCCESS;
   } else if (src == MPI_PROC_NULL) {
     *request = MPI_REQUEST_NULL;
     retval = MPI_SUCCESS;
-  } else if (src!=MPI_ANY_SOURCE && (src >= smpi_comm_group(comm)->size() || src <0)){
+  } else if (src!=MPI_ANY_SOURCE && (src >= comm->group()->size() || src <0)){
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
@@ -737,7 +737,7 @@ int PMPI_Irecv(void *buf, int count, MPI_Datatype datatype, int src, int tag, MP
   } else {
 
     int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
   } else {
 
     int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int src_traced = smpi_comm_group(comm)->index(src);
+    int src_traced = comm->group()->index(src);
 
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type = TRACING_IRECV;
 
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type = TRACING_IRECV;
@@ -777,7 +777,7 @@ int PMPI_Isend(void *buf, int count, MPI_Datatype datatype, int dst, int tag, MP
   } else if (dst == MPI_PROC_NULL) {
     *request = MPI_REQUEST_NULL;
     retval = MPI_SUCCESS;
   } else if (dst == MPI_PROC_NULL) {
     *request = MPI_REQUEST_NULL;
     retval = MPI_SUCCESS;
-  } else if (dst >= smpi_comm_group(comm)->size() || dst <0){
+  } else if (dst >= comm->group()->size() || dst <0){
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
@@ -787,7 +787,7 @@ int PMPI_Isend(void *buf, int count, MPI_Datatype datatype, int dst, int tag, MP
     retval = MPI_ERR_TAG;
   } else {
     int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     retval = MPI_ERR_TAG;
   } else {
     int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int dst_traced = smpi_comm_group(comm)->index(dst);
+    int dst_traced = comm->group()->index(dst);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type = TRACING_ISEND;
     extra->src = rank;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type = TRACING_ISEND;
     extra->src = rank;
@@ -826,7 +826,7 @@ int PMPI_Issend(void* buf, int count, MPI_Datatype datatype, int dst, int tag, M
   } else if (dst == MPI_PROC_NULL) {
     *request = MPI_REQUEST_NULL;
     retval = MPI_SUCCESS;
   } else if (dst == MPI_PROC_NULL) {
     *request = MPI_REQUEST_NULL;
     retval = MPI_SUCCESS;
-  } else if (dst >= smpi_comm_group(comm)->size() || dst <0){
+  } else if (dst >= comm->group()->size() || dst <0){
     retval = MPI_ERR_RANK;
   } else if ((count < 0)|| (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
     retval = MPI_ERR_RANK;
   } else if ((count < 0)|| (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
@@ -836,7 +836,7 @@ int PMPI_Issend(void* buf, int count, MPI_Datatype datatype, int dst, int tag, M
     retval = MPI_ERR_TAG;
   } else {
     int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     retval = MPI_ERR_TAG;
   } else {
     int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int dst_traced = smpi_comm_group(comm)->index(dst);
+    int dst_traced = comm->group()->index(dst);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type = TRACING_ISSEND;
     extra->src = rank;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type = TRACING_ISSEND;
     extra->src = rank;
@@ -874,7 +874,7 @@ int PMPI_Recv(void *buf, int count, MPI_Datatype datatype, int src, int tag, MPI
     smpi_empty_status(status);
     status->MPI_SOURCE = MPI_PROC_NULL;
     retval = MPI_SUCCESS;
     smpi_empty_status(status);
     status->MPI_SOURCE = MPI_PROC_NULL;
     retval = MPI_SUCCESS;
-  } else if (src!=MPI_ANY_SOURCE && (src >= smpi_comm_group(comm)->size() || src <0)){
+  } else if (src!=MPI_ANY_SOURCE && (src >= comm->group()->size() || src <0)){
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
@@ -884,7 +884,7 @@ int PMPI_Recv(void *buf, int count, MPI_Datatype datatype, int src, int tag, MPI
     retval = MPI_ERR_TAG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     retval = MPI_ERR_TAG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int src_traced         = smpi_comm_group(comm)->index(src);
+    int src_traced         = comm->group()->index(src);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_RECV;
     extra->src             = src_traced;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_RECV;
     extra->src             = src_traced;
@@ -902,7 +902,7 @@ int PMPI_Recv(void *buf, int count, MPI_Datatype datatype, int src, int tag, MPI
 
     // the src may not have been known at the beginning of the recv (MPI_ANY_SOURCE)
     if (status != MPI_STATUS_IGNORE) {
 
     // the src may not have been known at the beginning of the recv (MPI_ANY_SOURCE)
     if (status != MPI_STATUS_IGNORE) {
-      src_traced = smpi_comm_group(comm)->index(status->MPI_SOURCE);
+      src_traced = comm->group()->index(status->MPI_SOURCE);
       if (!TRACE_smpi_view_internals()) {
         TRACE_smpi_recv(rank, src_traced, rank, tag);
       }
       if (!TRACE_smpi_view_internals()) {
         TRACE_smpi_recv(rank, src_traced, rank, tag);
       }
@@ -924,7 +924,7 @@ int PMPI_Send(void *buf, int count, MPI_Datatype datatype, int dst, int tag, MPI
     retval = MPI_ERR_COMM;
   } else if (dst == MPI_PROC_NULL) {
     retval = MPI_SUCCESS;
     retval = MPI_ERR_COMM;
   } else if (dst == MPI_PROC_NULL) {
     retval = MPI_SUCCESS;
-  } else if (dst >= smpi_comm_group(comm)->size() || dst <0){
+  } else if (dst >= comm->group()->size() || dst <0){
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf == nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf == nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
@@ -934,7 +934,7 @@ int PMPI_Send(void *buf, int count, MPI_Datatype datatype, int dst, int tag, MPI
     retval = MPI_ERR_TAG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     retval = MPI_ERR_TAG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int dst_traced         = smpi_comm_group(comm)->index(dst);
+    int dst_traced         = comm->group()->index(dst);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type            = TRACING_SEND;
     extra->src             = rank;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type            = TRACING_SEND;
     extra->src             = rank;
@@ -970,7 +970,7 @@ int PMPI_Ssend(void* buf, int count, MPI_Datatype datatype, int dst, int tag, MP
     retval = MPI_ERR_COMM;
   } else if (dst == MPI_PROC_NULL) {
     retval = MPI_SUCCESS;
     retval = MPI_ERR_COMM;
   } else if (dst == MPI_PROC_NULL) {
     retval = MPI_SUCCESS;
-  } else if (dst >= smpi_comm_group(comm)->size() || dst <0){
+  } else if (dst >= comm->group()->size() || dst <0){
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
     retval = MPI_ERR_RANK;
   } else if ((count < 0) || (buf==nullptr && count > 0)) {
     retval = MPI_ERR_COUNT;
@@ -980,7 +980,7 @@ int PMPI_Ssend(void* buf, int count, MPI_Datatype datatype, int dst, int tag, MP
     retval = MPI_ERR_TAG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     retval = MPI_ERR_TAG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int dst_traced         = smpi_comm_group(comm)->index(dst);
+    int dst_traced         = comm->group()->index(dst);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type            = TRACING_SSEND;
     extra->src             = rank;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
     extra->type            = TRACING_SSEND;
     extra->src             = rank;
@@ -1020,8 +1020,8 @@ int PMPI_Sendrecv(void *sendbuf, int sendcount, MPI_Datatype sendtype, int dst,
     smpi_empty_status(status);
     status->MPI_SOURCE = MPI_PROC_NULL;
     retval             = MPI_SUCCESS;
     smpi_empty_status(status);
     status->MPI_SOURCE = MPI_PROC_NULL;
     retval             = MPI_SUCCESS;
-  }else if (dst >= smpi_comm_group(comm)->size() || dst <0 ||
-      (src!=MPI_ANY_SOURCE && (src >= smpi_comm_group(comm)->size() || src <0))){
+  }else if (dst >= comm->group()->size() || dst <0 ||
+      (src!=MPI_ANY_SOURCE && (src >= comm->group()->size() || src <0))){
     retval = MPI_ERR_RANK;
   } else if ((sendcount < 0 || recvcount<0) || 
       (sendbuf==nullptr && sendcount > 0) || (recvbuf==nullptr && recvcount>0)) {
     retval = MPI_ERR_RANK;
   } else if ((sendcount < 0 || recvcount<0) || 
       (sendbuf==nullptr && sendcount > 0) || (recvbuf==nullptr && recvcount>0)) {
@@ -1031,8 +1031,8 @@ int PMPI_Sendrecv(void *sendbuf, int sendcount, MPI_Datatype sendtype, int dst,
   } else {
 
   int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
   } else {
 
   int rank = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-  int dst_traced = smpi_comm_group(comm)->index(dst);
-  int src_traced = smpi_comm_group(comm)->index(src);
+  int dst_traced = comm->group()->index(dst);
+  int src_traced = comm->group()->index(src);
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_SENDRECV;
   extra->src = src_traced;
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_SENDRECV;
   extra->src = src_traced;
@@ -1215,7 +1215,7 @@ int PMPI_Wait(MPI_Request * request, MPI_Status * status)
     if (is_wait_for_receive) {
       if(src_traced==MPI_ANY_SOURCE)
         src_traced = (status!=MPI_STATUS_IGNORE) ?
     if (is_wait_for_receive) {
       if(src_traced==MPI_ANY_SOURCE)
         src_traced = (status!=MPI_STATUS_IGNORE) ?
-          smpi_comm_group(comm)->rank(status->MPI_SOURCE) :
+          comm->group()->rank(status->MPI_SOURCE) :
           src_traced;
       TRACE_smpi_recv(rank, src_traced, dst_traced, tag_traced);
     }
           src_traced;
       TRACE_smpi_recv(rank, src_traced, dst_traced, tag_traced);
     }
@@ -1265,7 +1265,7 @@ int PMPI_Waitany(int count, MPI_Request requests[], int *index, MPI_Status * sta
     if (is_wait_for_receive) {
       if(savedvals[*index].src==MPI_ANY_SOURCE)
         src_traced = (status != MPI_STATUSES_IGNORE)
     if (is_wait_for_receive) {
       if(savedvals[*index].src==MPI_ANY_SOURCE)
         src_traced = (status != MPI_STATUSES_IGNORE)
-                         ? smpi_comm_group(savedvals[*index].comm)->rank(status->MPI_SOURCE)
+                         ? savedvals[*index].comm->group()->rank(status->MPI_SOURCE)
                          : savedvals[*index].src;
       TRACE_smpi_recv(rank_traced, src_traced, dst_traced, savedvals[*index].tag);
     }
                          : savedvals[*index].src;
       TRACE_smpi_recv(rank_traced, src_traced, dst_traced, savedvals[*index].tag);
     }
@@ -1316,7 +1316,7 @@ int PMPI_Waitall(int count, MPI_Request requests[], MPI_Status status[])
       if (is_wait_for_receive) {
         if(src_traced==MPI_ANY_SOURCE)
         src_traced = (status!=MPI_STATUSES_IGNORE) ?
       if (is_wait_for_receive) {
         if(src_traced==MPI_ANY_SOURCE)
         src_traced = (status!=MPI_STATUSES_IGNORE) ?
-                          smpi_comm_group(savedvals[i].comm)->rank(status[i].MPI_SOURCE) : savedvals[i].src;
+                          savedvals[i].comm->group()->rank(status[i].MPI_SOURCE) : savedvals[i].src;
         TRACE_smpi_recv(rank_traced, src_traced, dst_traced,savedvals[i].tag);
       }
     }
         TRACE_smpi_recv(rank_traced, src_traced, dst_traced,savedvals[i].tag);
       }
     }
@@ -1371,7 +1371,7 @@ int PMPI_Bcast(void *buf, int count, MPI_Datatype datatype, int root, MPI_Comm c
     retval = MPI_ERR_ARG;
   } else {
     int rank        = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     retval = MPI_ERR_ARG;
   } else {
     int rank        = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int root_traced = smpi_comm_group(comm)->index(root);
+    int root_traced = comm->group()->index(root);
 
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_BCAST;
 
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_BCAST;
@@ -1383,7 +1383,7 @@ int PMPI_Bcast(void *buf, int count, MPI_Datatype datatype, int root, MPI_Comm c
       dt_size_send   = smpi_datatype_size(datatype);
     extra->send_size = count * dt_size_send;
     TRACE_smpi_collective_in(rank, root_traced, __FUNCTION__, extra);
       dt_size_send   = smpi_datatype_size(datatype);
     extra->send_size = count * dt_size_send;
     TRACE_smpi_collective_in(rank, root_traced, __FUNCTION__, extra);
-    if (smpi_comm_size(comm) > 1)
+    if (comm->size() > 1)
       mpi_coll_bcast_fun(buf, count, datatype, root, comm);
     retval = MPI_SUCCESS;
 
       mpi_coll_bcast_fun(buf, count, datatype, root, comm);
     retval = MPI_SUCCESS;
 
@@ -1427,21 +1427,21 @@ int PMPI_Gather(void *sendbuf, int sendcount, MPI_Datatype sendtype,void *recvbu
   if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
   } else if ((( sendbuf != MPI_IN_PLACE) && (sendtype == MPI_DATATYPE_NULL)) ||
   if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
   } else if ((( sendbuf != MPI_IN_PLACE) && (sendtype == MPI_DATATYPE_NULL)) ||
-            ((smpi_comm_rank(comm) == root) && (recvtype == MPI_DATATYPE_NULL))){
+            ((comm->rank() == root) && (recvtype == MPI_DATATYPE_NULL))){
     retval = MPI_ERR_TYPE;
     retval = MPI_ERR_TYPE;
-  } else if ((( sendbuf != MPI_IN_PLACE) && (sendcount <0)) || ((smpi_comm_rank(comm) == root) && (recvcount <0))){
+  } else if ((( sendbuf != MPI_IN_PLACE) && (sendcount <0)) || ((comm->rank() == root) && (recvcount <0))){
     retval = MPI_ERR_COUNT;
   } else {
 
     char* sendtmpbuf = static_cast<char*>(sendbuf);
     int sendtmpcount = sendcount;
     MPI_Datatype sendtmptype = sendtype;
     retval = MPI_ERR_COUNT;
   } else {
 
     char* sendtmpbuf = static_cast<char*>(sendbuf);
     int sendtmpcount = sendcount;
     MPI_Datatype sendtmptype = sendtype;
-    if( (smpi_comm_rank(comm) == root) && (sendbuf == MPI_IN_PLACE )) {
+    if( (comm->rank() == root) && (sendbuf == MPI_IN_PLACE )) {
       sendtmpcount=0;
       sendtmptype=recvtype;
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
       sendtmpcount=0;
       sendtmptype=recvtype;
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int root_traced        = smpi_comm_group(comm)->index(root);
+    int root_traced        = comm->group()->index(root);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_GATHER;
     extra->root            = root_traced;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_GATHER;
     extra->root            = root_traced;
@@ -1453,7 +1453,7 @@ int PMPI_Gather(void *sendbuf, int sendcount, MPI_Datatype sendtype,void *recvbu
     extra->send_size = sendtmpcount * dt_size_send;
     extra->datatype2 = encode_datatype(recvtype, &known);
     int dt_size_recv = 1;
     extra->send_size = sendtmpcount * dt_size_send;
     extra->datatype2 = encode_datatype(recvtype, &known);
     int dt_size_recv = 1;
-    if ((smpi_comm_rank(comm) == root) && known == 0)
+    if ((comm->rank() == root) && known == 0)
       dt_size_recv   = smpi_datatype_size(recvtype);
     extra->recv_size = recvcount * dt_size_recv;
 
       dt_size_recv   = smpi_datatype_size(recvtype);
     extra->recv_size = recvcount * dt_size_recv;
 
@@ -1479,7 +1479,7 @@ int PMPI_Gatherv(void *sendbuf, int sendcount, MPI_Datatype sendtype, void *recv
   if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
   } else if ((( sendbuf != MPI_IN_PLACE) && (sendtype == MPI_DATATYPE_NULL)) ||
   if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
   } else if ((( sendbuf != MPI_IN_PLACE) && (sendtype == MPI_DATATYPE_NULL)) ||
-            ((smpi_comm_rank(comm) == root) && (recvtype == MPI_DATATYPE_NULL))){
+            ((comm->rank() == root) && (recvtype == MPI_DATATYPE_NULL))){
     retval = MPI_ERR_TYPE;
   } else if (( sendbuf != MPI_IN_PLACE) && (sendcount <0)){
     retval = MPI_ERR_COUNT;
     retval = MPI_ERR_TYPE;
   } else if (( sendbuf != MPI_IN_PLACE) && (sendcount <0)){
     retval = MPI_ERR_COUNT;
@@ -1489,15 +1489,15 @@ int PMPI_Gatherv(void *sendbuf, int sendcount, MPI_Datatype sendtype, void *recv
     char* sendtmpbuf = static_cast<char*>(sendbuf);
     int sendtmpcount = sendcount;
     MPI_Datatype sendtmptype = sendtype;
     char* sendtmpbuf = static_cast<char*>(sendbuf);
     int sendtmpcount = sendcount;
     MPI_Datatype sendtmptype = sendtype;
-    if( (smpi_comm_rank(comm) == root) && (sendbuf == MPI_IN_PLACE )) {
+    if( (comm->rank() == root) && (sendbuf == MPI_IN_PLACE )) {
       sendtmpcount=0;
       sendtmptype=recvtype;
     }
 
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
       sendtmpcount=0;
       sendtmptype=recvtype;
     }
 
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int root_traced        = smpi_comm_group(comm)->index(root);
+    int root_traced        = comm->group()->index(root);
     int i                  = 0;
     int i                  = 0;
-    int size               = smpi_comm_size(comm);
+    int size               = comm->size();
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_GATHERV;
     extra->num_processes   = size;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_GATHERV;
     extra->num_processes   = size;
@@ -1512,7 +1512,7 @@ int PMPI_Gatherv(void *sendbuf, int sendcount, MPI_Datatype sendtype, void *recv
     int dt_size_recv = 1;
     if (known == 0)
       dt_size_recv = smpi_datatype_size(recvtype);
     int dt_size_recv = 1;
     if (known == 0)
       dt_size_recv = smpi_datatype_size(recvtype);
-    if ((smpi_comm_rank(comm) == root)) {
+    if ((comm->rank() == root)) {
       extra->recvcounts = xbt_new(int, size);
       for (i                 = 0; i < size; i++) // copy data to avoid bad free
         extra->recvcounts[i] = recvcounts[i] * dt_size_recv;
       extra->recvcounts = xbt_new(int, size);
       for (i                 = 0; i < size; i++) // copy data to avoid bad free
         extra->recvcounts[i] = recvcounts[i] * dt_size_recv;
@@ -1545,7 +1545,7 @@ int PMPI_Allgather(void *sendbuf, int sendcount, MPI_Datatype sendtype,
     retval = MPI_ERR_COUNT;
   } else {
     if(sendbuf == MPI_IN_PLACE) {
     retval = MPI_ERR_COUNT;
   } else {
     if(sendbuf == MPI_IN_PLACE) {
-      sendbuf=static_cast<char*>(recvbuf)+smpi_datatype_get_extent(recvtype)*recvcount*smpi_comm_rank(comm);
+      sendbuf=static_cast<char*>(recvbuf)+smpi_datatype_get_extent(recvtype)*recvcount*comm->rank();
       sendcount=recvcount;
       sendtype=recvtype;
     }
       sendcount=recvcount;
       sendtype=recvtype;
     }
@@ -1592,13 +1592,13 @@ int PMPI_Allgatherv(void *sendbuf, int sendcount, MPI_Datatype sendtype,
   } else {
 
     if(sendbuf == MPI_IN_PLACE) {
   } else {
 
     if(sendbuf == MPI_IN_PLACE) {
-      sendbuf=static_cast<char*>(recvbuf)+smpi_datatype_get_extent(recvtype)*displs[smpi_comm_rank(comm)];
-      sendcount=recvcounts[smpi_comm_rank(comm)];
+      sendbuf=static_cast<char*>(recvbuf)+smpi_datatype_get_extent(recvtype)*displs[comm->rank()];
+      sendcount=recvcounts[comm->rank()];
       sendtype=recvtype;
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     int i                  = 0;
       sendtype=recvtype;
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     int i                  = 0;
-    int size               = smpi_comm_size(comm);
+    int size               = comm->size();
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_ALLGATHERV;
     extra->num_processes   = size;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_ALLGATHERV;
     extra->num_processes   = size;
@@ -1636,11 +1636,11 @@ int PMPI_Scatter(void *sendbuf, int sendcount, MPI_Datatype sendtype,
 
   if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
 
   if (comm == MPI_COMM_NULL) {
     retval = MPI_ERR_COMM;
-  } else if (((smpi_comm_rank(comm) == root) && (!is_datatype_valid(sendtype))) ||
+  } else if (((comm->rank() == root) && (!is_datatype_valid(sendtype))) ||
              ((recvbuf != MPI_IN_PLACE) && (!is_datatype_valid(recvtype)))) {
     retval = MPI_ERR_TYPE;
   } else if ((sendbuf == recvbuf) ||
              ((recvbuf != MPI_IN_PLACE) && (!is_datatype_valid(recvtype)))) {
     retval = MPI_ERR_TYPE;
   } else if ((sendbuf == recvbuf) ||
-      ((smpi_comm_rank(comm)==root) && sendcount>0 && (sendbuf == nullptr))){
+      ((comm->rank()==root) && sendcount>0 && (sendbuf == nullptr))){
     retval = MPI_ERR_BUFFER;
   }else {
 
     retval = MPI_ERR_BUFFER;
   }else {
 
@@ -1649,14 +1649,14 @@ int PMPI_Scatter(void *sendbuf, int sendcount, MPI_Datatype sendtype,
       recvcount = sendcount;
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
       recvcount = sendcount;
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int root_traced        = smpi_comm_group(comm)->index(root);
+    int root_traced        = comm->group()->index(root);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_SCATTER;
     extra->root            = root_traced;
     int known              = 0;
     extra->datatype1       = encode_datatype(sendtype, &known);
     int dt_size_send       = 1;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_SCATTER;
     extra->root            = root_traced;
     int known              = 0;
     extra->datatype1       = encode_datatype(sendtype, &known);
     int dt_size_send       = 1;
-    if ((smpi_comm_rank(comm) == root) && known == 0)
+    if ((comm->rank() == root) && known == 0)
       dt_size_send   = smpi_datatype_size(sendtype);
     extra->send_size = sendcount * dt_size_send;
     extra->datatype2 = encode_datatype(recvtype, &known);
       dt_size_send   = smpi_datatype_size(sendtype);
     extra->send_size = sendcount * dt_size_send;
     extra->datatype2 = encode_datatype(recvtype, &known);
@@ -1686,18 +1686,18 @@ int PMPI_Scatterv(void *sendbuf, int *sendcounts, int *displs,
     retval = MPI_ERR_COMM;
   } else if (sendcounts == nullptr || displs == nullptr) {
     retval = MPI_ERR_ARG;
     retval = MPI_ERR_COMM;
   } else if (sendcounts == nullptr || displs == nullptr) {
     retval = MPI_ERR_ARG;
-  } else if (((smpi_comm_rank(comm) == root) && (sendtype == MPI_DATATYPE_NULL)) ||
+  } else if (((comm->rank() == root) && (sendtype == MPI_DATATYPE_NULL)) ||
              ((recvbuf != MPI_IN_PLACE) && (recvtype == MPI_DATATYPE_NULL))) {
     retval = MPI_ERR_TYPE;
   } else {
     if (recvbuf == MPI_IN_PLACE) {
       recvtype  = sendtype;
              ((recvbuf != MPI_IN_PLACE) && (recvtype == MPI_DATATYPE_NULL))) {
     retval = MPI_ERR_TYPE;
   } else {
     if (recvbuf == MPI_IN_PLACE) {
       recvtype  = sendtype;
-      recvcount = sendcounts[smpi_comm_rank(comm)];
+      recvcount = sendcounts[comm->rank()];
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     }
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int root_traced        = smpi_comm_group(comm)->index(root);
+    int root_traced        = comm->group()->index(root);
     int i                  = 0;
     int i                  = 0;
-    int size               = smpi_comm_size(comm);
+    int size               = comm->size();
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_SCATTERV;
     extra->num_processes   = size;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_SCATTERV;
     extra->num_processes   = size;
@@ -1707,7 +1707,7 @@ int PMPI_Scatterv(void *sendbuf, int *sendcounts, int *displs,
     int dt_size_send       = 1;
     if (known == 0)
       dt_size_send = smpi_datatype_size(sendtype);
     int dt_size_send       = 1;
     if (known == 0)
       dt_size_send = smpi_datatype_size(sendtype);
-    if ((smpi_comm_rank(comm) == root)) {
+    if ((comm->rank() == root)) {
       extra->sendcounts = xbt_new(int, size);
       for (i                 = 0; i < size; i++) // copy data to avoid bad free
         extra->sendcounts[i] = sendcounts[i] * dt_size_send;
       extra->sendcounts = xbt_new(int, size);
       for (i                 = 0; i < size; i++) // copy data to avoid bad free
         extra->sendcounts[i] = sendcounts[i] * dt_size_send;
@@ -1741,7 +1741,7 @@ int PMPI_Reduce(void *sendbuf, void *recvbuf, int count, MPI_Datatype datatype,
     retval = MPI_ERR_ARG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     retval = MPI_ERR_ARG;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
-    int root_traced        = smpi_comm_group(comm)->index(root);
+    int root_traced        = comm->group()->index(root);
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_REDUCE;
     int known              = 0;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_REDUCE;
     int known              = 0;
@@ -1912,7 +1912,7 @@ int PMPI_Reduce_scatter(void *sendbuf, void *recvbuf, int *recvcounts, MPI_Datat
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     int i                  = 0;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     int i                  = 0;
-    int size               = smpi_comm_size(comm);
+    int size               = comm->size();
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_REDUCE_SCATTER;
     extra->num_processes   = size;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_REDUCE_SCATTER;
     extra->num_processes   = size;
@@ -1963,7 +1963,7 @@ int PMPI_Reduce_scatter_block(void *sendbuf, void *recvbuf, int recvcount,
   } else if (recvcount < 0) {
     retval = MPI_ERR_ARG;
   } else {
   } else if (recvcount < 0) {
     retval = MPI_ERR_ARG;
   } else {
-    int count = smpi_comm_size(comm);
+    int count = comm->size();
 
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
 
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
@@ -2022,8 +2022,8 @@ int PMPI_Alltoall(void* sendbuf, int sendcount, MPI_Datatype sendtype, void* rec
     int sendtmpcount         = sendcount;
     MPI_Datatype sendtmptype = sendtype;
     if (sendbuf == MPI_IN_PLACE) {
     int sendtmpcount         = sendcount;
     MPI_Datatype sendtmptype = sendtype;
     if (sendbuf == MPI_IN_PLACE) {
-      sendtmpbuf = static_cast<void*>(xbt_malloc(recvcount * smpi_comm_size(comm) * smpi_datatype_size(recvtype)));
-      memcpy(sendtmpbuf, recvbuf, recvcount * smpi_comm_size(comm) * smpi_datatype_size(recvtype));
+      sendtmpbuf = static_cast<void*>(xbt_malloc(recvcount * comm->size() * smpi_datatype_size(recvtype)));
+      memcpy(sendtmpbuf, recvbuf, recvcount * comm->size() * smpi_datatype_size(recvtype));
       sendtmpcount = recvcount;
       sendtmptype  = recvtype;
     }
       sendtmpcount = recvcount;
       sendtmptype  = recvtype;
     }
@@ -2071,7 +2071,7 @@ int PMPI_Alltoallv(void* sendbuf, int* sendcounts, int* senddisps, MPI_Datatype
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     int i                  = 0;
   } else {
     int rank               = comm != MPI_COMM_NULL ? smpi_process_index() : -1;
     int i                  = 0;
-    int size               = smpi_comm_size(comm);
+    int size               = comm->size();
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_ALLTOALLV;
     extra->send_size       = 0;
     instr_extra_data extra = xbt_new0(s_instr_extra_data_t, 1);
     extra->type            = TRACING_ALLTOALLV;
     extra->send_size       = 0;
@@ -2312,7 +2312,7 @@ int PMPI_Cart_create(MPI_Comm comm_old, int ndims, int* dims, int* periodic, int
 }
 
 int PMPI_Cart_rank(MPI_Comm comm, int* coords, int* rank) {
 }
 
 int PMPI_Cart_rank(MPI_Comm comm, int* coords, int* rank) {
-  if(comm == MPI_COMM_NULL || smpi_comm_topo(comm) == nullptr) {
+  if(comm == MPI_COMM_NULL || comm->topo() == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if (coords == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if (coords == nullptr) {
@@ -2322,7 +2322,7 @@ int PMPI_Cart_rank(MPI_Comm comm, int* coords, int* rank) {
 }
 
 int PMPI_Cart_shift(MPI_Comm comm, int direction, int displ, int* source, int* dest) {
 }
 
 int PMPI_Cart_shift(MPI_Comm comm, int direction, int displ, int* source, int* dest) {
-  if(comm == MPI_COMM_NULL || smpi_comm_topo(comm) == nullptr) {
+  if(comm == MPI_COMM_NULL || comm->topo() == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if (source == nullptr || dest == nullptr || direction < 0 ) {
     return MPI_ERR_TOPOLOGY;
   }
   if (source == nullptr || dest == nullptr || direction < 0 ) {
@@ -2332,10 +2332,10 @@ int PMPI_Cart_shift(MPI_Comm comm, int direction, int displ, int* source, int* d
 }
 
 int PMPI_Cart_coords(MPI_Comm comm, int rank, int maxdims, int* coords) {
 }
 
 int PMPI_Cart_coords(MPI_Comm comm, int rank, int maxdims, int* coords) {
-  if(comm == MPI_COMM_NULL || smpi_comm_topo(comm) == nullptr) {
+  if(comm == MPI_COMM_NULL || comm->topo() == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
     return MPI_ERR_TOPOLOGY;
   }
-  if (rank < 0 || rank >= smpi_comm_size(comm)) {
+  if (rank < 0 || rank >= comm->size()) {
     return MPI_ERR_RANK;
   }
   if (maxdims <= 0) {
     return MPI_ERR_RANK;
   }
   if (maxdims <= 0) {
@@ -2348,7 +2348,7 @@ int PMPI_Cart_coords(MPI_Comm comm, int rank, int maxdims, int* coords) {
 }
 
 int PMPI_Cart_get(MPI_Comm comm, int maxdims, int* dims, int* periods, int* coords) {
 }
 
 int PMPI_Cart_get(MPI_Comm comm, int maxdims, int* dims, int* periods, int* coords) {
-  if(comm == nullptr || smpi_comm_topo(comm) == nullptr) {
+  if(comm == nullptr || comm->topo() == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if(maxdims <= 0 || dims == nullptr || periods == nullptr || coords == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if(maxdims <= 0 || dims == nullptr || periods == nullptr || coords == nullptr) {
@@ -2358,7 +2358,7 @@ int PMPI_Cart_get(MPI_Comm comm, int maxdims, int* dims, int* periods, int* coor
 }
 
 int PMPI_Cartdim_get(MPI_Comm comm, int* ndims) {
 }
 
 int PMPI_Cartdim_get(MPI_Comm comm, int* ndims) {
-  if (comm == MPI_COMM_NULL || smpi_comm_topo(comm) == nullptr) {
+  if (comm == MPI_COMM_NULL || comm->topo() == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if (ndims == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if (ndims == nullptr) {
@@ -2379,7 +2379,7 @@ int PMPI_Dims_create(int nnodes, int ndims, int* dims) {
 }
 
 int PMPI_Cart_sub(MPI_Comm comm, int* remain_dims, MPI_Comm* comm_new) {
 }
 
 int PMPI_Cart_sub(MPI_Comm comm, int* remain_dims, MPI_Comm* comm_new) {
-  if(comm == MPI_COMM_NULL || smpi_comm_topo(comm) == nullptr) {
+  if(comm == MPI_COMM_NULL || comm->topo() == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if (comm_new == nullptr) {
     return MPI_ERR_TOPOLOGY;
   }
   if (comm_new == nullptr) {
@@ -2760,7 +2760,7 @@ int PMPI_Attr_delete(MPI_Comm comm, int keyval) {
   else if (comm==MPI_COMM_NULL)
     return MPI_ERR_COMM;
   else
   else if (comm==MPI_COMM_NULL)
     return MPI_ERR_COMM;
   else
-    return smpi_comm_attr_delete(comm, keyval);
+    return comm->attr_delete(keyval);
 }
 
 int PMPI_Attr_get(MPI_Comm comm, int keyval, void* attr_value, int* flag) {
 }
 
 int PMPI_Attr_get(MPI_Comm comm, int keyval, void* attr_value, int* flag) {
@@ -2798,7 +2798,7 @@ int PMPI_Attr_get(MPI_Comm comm, int keyval, void* attr_value, int* flag) {
     *static_cast<int**>(attr_value) = &one;
     return MPI_SUCCESS;
   default:
     *static_cast<int**>(attr_value) = &one;
     return MPI_SUCCESS;
   default:
-    return smpi_comm_attr_get(comm, keyval, attr_value, flag);
+    return comm->attr_get(keyval, attr_value, flag);
   }
 }
 
   }
 }
 
@@ -2809,7 +2809,7 @@ int PMPI_Attr_put(MPI_Comm comm, int keyval, void* attr_value) {
   else if (comm==MPI_COMM_NULL)
     return MPI_ERR_COMM;
   else
   else if (comm==MPI_COMM_NULL)
     return MPI_ERR_COMM;
   else
-  return smpi_comm_attr_put(comm, keyval, attr_value);
+  return comm->attr_put(keyval, attr_value);
 }
 
 int PMPI_Comm_get_attr (MPI_Comm comm, int comm_keyval, void *attribute_val, int *flag)
 }
 
 int PMPI_Comm_get_attr (MPI_Comm comm, int comm_keyval, void *attribute_val, int *flag)
index 38def2b..93404cc 100644 (file)
@@ -218,7 +218,7 @@ static void action_send(const char *const *action)
 
   int rank = smpi_process_index();
 
 
   int rank = smpi_process_index();
 
-  int dst_traced = smpi_comm_group(MPI_COMM_WORLD)->rank(to);
+  int dst_traced = MPI_COMM_WORLD->group()->rank(to);
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_SEND;
   extra->send_size = size;
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_SEND;
   extra->send_size = size;
@@ -249,7 +249,7 @@ static void action_Isend(const char *const *action)
     MPI_CURRENT_TYPE= MPI_DEFAULT_TYPE;
 
   int rank = smpi_process_index();
     MPI_CURRENT_TYPE= MPI_DEFAULT_TYPE;
 
   int rank = smpi_process_index();
-  int dst_traced = smpi_comm_group(MPI_COMM_WORLD)->rank(to);
+  int dst_traced = MPI_COMM_WORLD->group()->rank(to);
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_ISEND;
   extra->send_size = size;
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_ISEND;
   extra->send_size = size;
@@ -283,7 +283,7 @@ static void action_recv(const char *const *action) {
     MPI_CURRENT_TYPE= MPI_DEFAULT_TYPE;
 
   int rank = smpi_process_index();
     MPI_CURRENT_TYPE= MPI_DEFAULT_TYPE;
 
   int rank = smpi_process_index();
-  int src_traced = smpi_comm_group(MPI_COMM_WORLD)->rank(from);
+  int src_traced = MPI_COMM_WORLD->group()->rank(from);
 
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_RECV;
 
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_RECV;
@@ -322,7 +322,7 @@ static void action_Irecv(const char *const *action)
     MPI_CURRENT_TYPE= MPI_DEFAULT_TYPE;
 
   int rank = smpi_process_index();
     MPI_CURRENT_TYPE= MPI_DEFAULT_TYPE;
 
   int rank = smpi_process_index();
-  int src_traced = smpi_comm_group(MPI_COMM_WORLD)->rank(from);
+  int src_traced = MPI_COMM_WORLD->group()->rank(from);
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_IRECV;
   extra->send_size = size;
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_IRECV;
   extra->send_size = size;
@@ -388,9 +388,9 @@ static void action_wait(const char *const *action){
     return;
   }
 
     return;
   }
 
-  int rank = request->comm != MPI_COMM_NULL ? smpi_comm_rank(request->comm) : -1;
+  int rank = request->comm != MPI_COMM_NULL ? request->comm->rank() : -1;
 
 
-  MPI_Group group = smpi_comm_group(request->comm);
+  MPI_Group group = request->comm->group();
   int src_traced = group->rank(request->src);
   int dst_traced = group->rank(request->dst);
   int is_wait_for_receive = request->recv;
   int src_traced = group->rank(request->src);
   int dst_traced = group->rank(request->dst);
   int is_wait_for_receive = request->recv;
@@ -470,7 +470,7 @@ static void action_bcast(const char *const *action)
   }
 
   int rank = smpi_process_index();
   }
 
   int rank = smpi_process_index();
-  int root_traced = smpi_comm_group(MPI_COMM_WORLD)->index(root);
+  int root_traced = MPI_COMM_WORLD->group()->index(root);
 
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_BCAST;
 
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_BCAST;
@@ -502,7 +502,7 @@ static void action_reduce(const char *const *action)
   }
 
   int rank = smpi_process_index();
   }
 
   int rank = smpi_process_index();
-  int root_traced = smpi_comm_group(MPI_COMM_WORLD)->rank(root);
+  int root_traced = MPI_COMM_WORLD->group()->rank(root);
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_REDUCE;
   extra->send_size = comm_size;
   instr_extra_data extra = xbt_new0(s_instr_extra_data_t,1);
   extra->type = TRACING_REDUCE;
   extra->send_size = comm_size;
@@ -552,7 +552,7 @@ static void action_allReduce(const char *const *action) {
 static void action_allToAll(const char *const *action) {
   CHECK_ACTION_PARAMS(action, 2, 2) //two mandatory (send and recv volumes) and two optional (corresponding datatypes)
   double clock = smpi_process_simulated_elapsed();
 static void action_allToAll(const char *const *action) {
   CHECK_ACTION_PARAMS(action, 2, 2) //two mandatory (send and recv volumes) and two optional (corresponding datatypes)
   double clock = smpi_process_simulated_elapsed();
-  int comm_size = smpi_comm_size(MPI_COMM_WORLD);
+  int comm_size = MPI_COMM_WORLD->size();
   int send_size = parse_double(action[2]);
   int recv_size = parse_double(action[3]);
   MPI_Datatype MPI_CURRENT_TYPE2 = MPI_DEFAULT_TYPE;
   int send_size = parse_double(action[2]);
   int recv_size = parse_double(action[3]);
   MPI_Datatype MPI_CURRENT_TYPE2 = MPI_DEFAULT_TYPE;
@@ -595,7 +595,7 @@ static void action_gather(const char *const *action) {
   */
   CHECK_ACTION_PARAMS(action, 2, 3)
   double clock = smpi_process_simulated_elapsed();
   */
   CHECK_ACTION_PARAMS(action, 2, 3)
   double clock = smpi_process_simulated_elapsed();
-  int comm_size = smpi_comm_size(MPI_COMM_WORLD);
+  int comm_size = MPI_COMM_WORLD->size();
   int send_size = parse_double(action[2]);
   int recv_size = parse_double(action[3]);
   MPI_Datatype MPI_CURRENT_TYPE2 = MPI_DEFAULT_TYPE;
   int send_size = parse_double(action[2]);
   int recv_size = parse_double(action[3]);
   MPI_Datatype MPI_CURRENT_TYPE2 = MPI_DEFAULT_TYPE;
@@ -610,7 +610,7 @@ static void action_gather(const char *const *action) {
   int root=0;
   if(action[4])
     root=atoi(action[4]);
   int root=0;
   if(action[4])
     root=atoi(action[4]);
-  int rank = smpi_comm_rank(MPI_COMM_WORLD);
+  int rank = MPI_COMM_WORLD->rank();
 
   if(rank==root)
     recv = smpi_get_tmp_recvbuffer(recv_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
 
   if(rank==root)
     recv = smpi_get_tmp_recvbuffer(recv_size*comm_size* smpi_datatype_size(MPI_CURRENT_TYPE2));
@@ -642,7 +642,7 @@ static void action_gatherv(const char *const *action) {
        5) 0 is the recv datatype id, see decode_datatype()
   */
   double clock = smpi_process_simulated_elapsed();
        5) 0 is the recv datatype id, see decode_datatype()
   */
   double clock = smpi_process_simulated_elapsed();
-  int comm_size = smpi_comm_size(MPI_COMM_WORLD);
+  int comm_size = MPI_COMM_WORLD->size();
   CHECK_ACTION_PARAMS(action, comm_size+1, 2)
   int send_size = parse_double(action[2]);
   int disps[comm_size];
   CHECK_ACTION_PARAMS(action, comm_size+1, 2)
   int send_size = parse_double(action[2]);
   int disps[comm_size];
@@ -665,7 +665,7 @@ static void action_gatherv(const char *const *action) {
   }
 
   int root=atoi(action[3+comm_size]);
   }
 
   int root=atoi(action[3+comm_size]);
-  int rank = smpi_comm_rank(MPI_COMM_WORLD);
+  int rank = MPI_COMM_WORLD->rank();
 
   if(rank==root)
     recv = smpi_get_tmp_recvbuffer(recv_sum* smpi_datatype_size(MPI_CURRENT_TYPE2));
 
   if(rank==root)
     recv = smpi_get_tmp_recvbuffer(recv_sum* smpi_datatype_size(MPI_CURRENT_TYPE2));
@@ -698,7 +698,7 @@ static void action_reducescatter(const char *const *action) {
       3) The last value corresponds to the datatype, see decode_datatype().
 */
   double clock = smpi_process_simulated_elapsed();
       3) The last value corresponds to the datatype, see decode_datatype().
 */
   double clock = smpi_process_simulated_elapsed();
-  int comm_size = smpi_comm_size(MPI_COMM_WORLD);
+  int comm_size = MPI_COMM_WORLD->size();
   CHECK_ACTION_PARAMS(action, comm_size+1, 1)
   int comp_size = parse_double(action[2+comm_size]);
   int recvcounts[comm_size];
   CHECK_ACTION_PARAMS(action, comm_size+1, 1)
   int comp_size = parse_double(action[2+comm_size]);
   int recvcounts[comm_size];
@@ -768,7 +768,7 @@ static void action_allgather(const char *const *action) {
   extra->recv_size= recvcount;
   extra->datatype1 = encode_datatype(MPI_CURRENT_TYPE, nullptr);
   extra->datatype2 = encode_datatype(MPI_CURRENT_TYPE2, nullptr);
   extra->recv_size= recvcount;
   extra->datatype1 = encode_datatype(MPI_CURRENT_TYPE, nullptr);
   extra->datatype2 = encode_datatype(MPI_CURRENT_TYPE2, nullptr);
-  extra->num_processes = smpi_comm_size(MPI_COMM_WORLD);
+  extra->num_processes = MPI_COMM_WORLD->size();
 
   TRACE_smpi_collective_in(rank, -1, __FUNCTION__,extra);
 
 
   TRACE_smpi_collective_in(rank, -1, __FUNCTION__,extra);
 
@@ -788,7 +788,7 @@ static void action_allgatherv(const char *const *action) {
   */
   double clock = smpi_process_simulated_elapsed();
 
   */
   double clock = smpi_process_simulated_elapsed();
 
-  int comm_size = smpi_comm_size(MPI_COMM_WORLD);
+  int comm_size = MPI_COMM_WORLD->size();
   CHECK_ACTION_PARAMS(action, comm_size+1, 2)
   int sendcount=atoi(action[2]);
   int recvcounts[comm_size];
   CHECK_ACTION_PARAMS(action, comm_size+1, 2)
   int sendcount=atoi(action[2]);
   int recvcounts[comm_size];
@@ -842,7 +842,7 @@ static void action_allToAllv(const char *const *action) {
   */
   double clock = smpi_process_simulated_elapsed();
 
   */
   double clock = smpi_process_simulated_elapsed();
 
-  int comm_size = smpi_comm_size(MPI_COMM_WORLD);
+  int comm_size = MPI_COMM_WORLD->size();
   CHECK_ACTION_PARAMS(action, 2*comm_size+2, 2)
   int sendcounts[comm_size];
   int recvcounts[comm_size];
   CHECK_ACTION_PARAMS(action, 2*comm_size+2, 2)
   int sendcounts[comm_size];
   int recvcounts[comm_size];
index 42de984..2345f55 100644 (file)
@@ -29,8 +29,8 @@ typedef struct s_smpi_mpi_win{
 
 
 MPI_Win smpi_mpi_win_create( void *base, MPI_Aint size, int disp_unit, MPI_Info info, MPI_Comm comm){
 
 
 MPI_Win smpi_mpi_win_create( void *base, MPI_Aint size, int disp_unit, MPI_Info info, MPI_Comm comm){
-  int comm_size = smpi_comm_size(comm);
-  int rank      = smpi_comm_rank(comm);
+  int comm_size = comm->size();
+  int rank      = comm->rank();
   XBT_DEBUG("Creating window");
 
   MPI_Win win    = xbt_new(s_smpi_mpi_win_t, 1);
   XBT_DEBUG("Creating window");
 
   MPI_Win win    = xbt_new(s_smpi_mpi_win_t, 1);
@@ -78,7 +78,7 @@ int smpi_mpi_win_free( MPI_Win* win){
   }
 
   mpi_coll_barrier_fun((*win)->comm);
   }
 
   mpi_coll_barrier_fun((*win)->comm);
-  int rank=smpi_comm_rank((*win)->comm);
+  int rank=(*win)->comm->rank();
   if(rank == 0)
     MSG_barrier_destroy((*win)->bar);
   xbt_mutex_destroy((*win)->mut);
   if(rank == 0)
     MSG_barrier_destroy((*win)->bar);
   xbt_mutex_destroy((*win)->mut);
@@ -99,7 +99,7 @@ void smpi_mpi_win_get_name(MPI_Win win, char* name, int* length){
 
 void smpi_mpi_win_get_group(MPI_Win win, MPI_Group* group){
   if(win->comm != MPI_COMM_NULL){
 
 void smpi_mpi_win_get_group(MPI_Win win, MPI_Group* group){
   if(win->comm != MPI_COMM_NULL){
-    *group = smpi_comm_group(win->comm);
+    *group = win->comm->group();
   } else {
     *group = MPI_GROUP_NULL;
   }
   } else {
     *group = MPI_GROUP_NULL;
   }
@@ -157,14 +157,14 @@ int smpi_mpi_put( void *origin_addr, int origin_count, MPI_Datatype origin_datat
   void* recv_addr = static_cast<void*> ( static_cast<char*>(recv_win->base) + target_disp * recv_win->disp_unit);
   XBT_DEBUG("Entering MPI_Put to %d", target_rank);
 
   void* recv_addr = static_cast<void*> ( static_cast<char*>(recv_win->base) + target_disp * recv_win->disp_unit);
   XBT_DEBUG("Entering MPI_Put to %d", target_rank);
 
-  if(target_rank != smpi_comm_rank(win->comm)){
+  if(target_rank != win->comm->rank()){
     //prepare send_request
     MPI_Request sreq = smpi_rma_send_init(origin_addr, origin_count, origin_datatype, smpi_process_index(),
     //prepare send_request
     MPI_Request sreq = smpi_rma_send_init(origin_addr, origin_count, origin_datatype, smpi_process_index(),
-        smpi_comm_group(win->comm)->index(target_rank), SMPI_RMA_TAG+1, win->comm, MPI_OP_NULL);
+        win->comm->group()->index(target_rank), SMPI_RMA_TAG+1, win->comm, MPI_OP_NULL);
 
     //prepare receiver request
     MPI_Request rreq = smpi_rma_recv_init(recv_addr, target_count, target_datatype, smpi_process_index(),
 
     //prepare receiver request
     MPI_Request rreq = smpi_rma_recv_init(recv_addr, target_count, target_datatype, smpi_process_index(),
-        smpi_comm_group(win->comm)->index(target_rank), SMPI_RMA_TAG+1, recv_win->comm, MPI_OP_NULL);
+        win->comm->group()->index(target_rank), SMPI_RMA_TAG+1, recv_win->comm, MPI_OP_NULL);
 
     //push request to receiver's win
     xbt_mutex_acquire(recv_win->mut);
 
     //push request to receiver's win
     xbt_mutex_acquire(recv_win->mut);
@@ -195,15 +195,15 @@ int smpi_mpi_get( void *origin_addr, int origin_count, MPI_Datatype origin_datat
   void* send_addr = static_cast<void*>(static_cast<char*>(send_win->base) + target_disp * send_win->disp_unit);
   XBT_DEBUG("Entering MPI_Get from %d", target_rank);
 
   void* send_addr = static_cast<void*>(static_cast<char*>(send_win->base) + target_disp * send_win->disp_unit);
   XBT_DEBUG("Entering MPI_Get from %d", target_rank);
 
-  if(target_rank != smpi_comm_rank(win->comm)){
+  if(target_rank != win->comm->rank()){
     //prepare send_request
     MPI_Request sreq = smpi_rma_send_init(send_addr, target_count, target_datatype,
     //prepare send_request
     MPI_Request sreq = smpi_rma_send_init(send_addr, target_count, target_datatype,
-        smpi_comm_group(win->comm)->index(target_rank), smpi_process_index(), SMPI_RMA_TAG+2, send_win->comm,
+        win->comm->group()->index(target_rank), smpi_process_index(), SMPI_RMA_TAG+2, send_win->comm,
         MPI_OP_NULL);
 
     //prepare receiver request
     MPI_Request rreq = smpi_rma_recv_init(origin_addr, origin_count, origin_datatype,
         MPI_OP_NULL);
 
     //prepare receiver request
     MPI_Request rreq = smpi_rma_recv_init(origin_addr, origin_count, origin_datatype,
-        smpi_comm_group(win->comm)->index(target_rank), smpi_process_index(), SMPI_RMA_TAG+2, win->comm,
+        win->comm->group()->index(target_rank), smpi_process_index(), SMPI_RMA_TAG+2, win->comm,
         MPI_OP_NULL);
 
     //start the send, with another process than us as sender. 
         MPI_OP_NULL);
 
     //start the send, with another process than us as sender. 
@@ -241,11 +241,11 @@ int smpi_mpi_accumulate( void *origin_addr, int origin_count, MPI_Datatype origi
     //As the tag will be used for ordering of the operations, add count to it
     //prepare send_request
     MPI_Request sreq = smpi_rma_send_init(origin_addr, origin_count, origin_datatype,
     //As the tag will be used for ordering of the operations, add count to it
     //prepare send_request
     MPI_Request sreq = smpi_rma_send_init(origin_addr, origin_count, origin_datatype,
-        smpi_process_index(), smpi_comm_group(win->comm)->index(target_rank), SMPI_RMA_TAG+3+win->count, win->comm, op);
+        smpi_process_index(), win->comm->group()->index(target_rank), SMPI_RMA_TAG+3+win->count, win->comm, op);
 
     //prepare receiver request
     MPI_Request rreq = smpi_rma_recv_init(recv_addr, target_count, target_datatype,
 
     //prepare receiver request
     MPI_Request rreq = smpi_rma_recv_init(recv_addr, target_count, target_datatype,
-        smpi_process_index(), smpi_comm_group(win->comm)->index(target_rank), SMPI_RMA_TAG+3+win->count, recv_win->comm, op);
+        smpi_process_index(), win->comm->group()->index(target_rank), SMPI_RMA_TAG+3+win->count, recv_win->comm, op);
 
     win->count++;
     //push request to receiver's win
 
     win->count++;
     //push request to receiver's win
index fc77a2e..b9a3786 100644 (file)
@@ -125,7 +125,7 @@ int smpi_mpi_cart_create(MPI_Comm comm_old, int ndims, int dims[], int periods[]
   MPI_Group oldGroup;
   int nranks;
 
   MPI_Group oldGroup;
   int nranks;
 
-  int rank = smpi_comm_rank(comm_old);
+  int rank = comm_old->rank();
 
   int newSize = 1;
   if(ndims != 0) {
 
   int newSize = 1;
   if(ndims != 0) {
@@ -137,7 +137,7 @@ int smpi_mpi_cart_create(MPI_Comm comm_old, int ndims, int dims[], int periods[]
       return retval;
     }
     newCart = smpi_cart_topo_create(ndims);
       return retval;
     }
     newCart = smpi_cart_topo_create(ndims);
-    oldGroup = smpi_comm_group(comm_old);
+    oldGroup = comm_old->group();
     newGroup = new simgrid::SMPI::Group(newSize);
     for (int i = 0 ; i < newSize ; i++) {
       newGroup->set_mapping(oldGroup->index(i), i);
     newGroup = new simgrid::SMPI::Group(newSize);
     for (int i = 0 ; i < newSize ; i++) {
       newGroup->set_mapping(oldGroup->index(i), i);
@@ -156,11 +156,11 @@ int smpi_mpi_cart_create(MPI_Comm comm_old, int ndims, int dims[], int periods[]
       rank = rank % nranks;
     }
 
       rank = rank % nranks;
     }
 
-    *comm_cart = smpi_comm_new(newGroup, newCart);
+    *comm_cart = new simgrid::SMPI::Comm(newGroup, newCart);
   } else {
     if (rank == 0) {
       newCart = smpi_cart_topo_create(ndims);
   } else {
     if (rank == 0) {
       newCart = smpi_cart_topo_create(ndims);
-      *comm_cart = smpi_comm_new(new simgrid::SMPI::Group(smpi_comm_group(MPI_COMM_SELF)), newCart);
+      *comm_cart = new simgrid::SMPI::Comm(new simgrid::SMPI::Group(MPI_COMM_SELF->group()), newCart);
     } else {
       *comm_cart = MPI_COMM_NULL;
     }
     } else {
       *comm_cart = MPI_COMM_NULL;
     }
@@ -169,7 +169,7 @@ int smpi_mpi_cart_create(MPI_Comm comm_old, int ndims, int dims[], int periods[]
 }
 
 int smpi_mpi_cart_sub(MPI_Comm comm, const int remain_dims[], MPI_Comm *newcomm) {
 }
 
 int smpi_mpi_cart_sub(MPI_Comm comm, const int remain_dims[], MPI_Comm *newcomm) {
-  MPI_Topology oldTopo = smpi_comm_topo(comm);
+  MPI_Topology oldTopo = comm->topo();
   int oldNDims = oldTopo->topo.cart->ndims;
   int j = 0;
   int *newDims = nullptr;
   int oldNDims = oldTopo->topo.cart->ndims;
   int j = 0;
   int *newDims = nullptr;
@@ -201,7 +201,7 @@ int smpi_mpi_cart_sub(MPI_Comm comm, const int remain_dims[], MPI_Comm *newcomm)
 }
 
 int smpi_mpi_cart_coords(MPI_Comm comm, int rank, int maxdims, int coords[]) {
 }
 
 int smpi_mpi_cart_coords(MPI_Comm comm, int rank, int maxdims, int coords[]) {
-  MPI_Topology topo = smpi_comm_topo(comm);
+  MPI_Topology topo = comm->topo();
   int nnodes = topo->topo.cart->nnodes;
   for (int i = 0; i< topo->topo.cart->ndims; i++ ) {
     nnodes    = nnodes / topo->topo.cart->dims[i];
   int nnodes = topo->topo.cart->nnodes;
   for (int i = 0; i< topo->topo.cart->ndims; i++ ) {
     nnodes    = nnodes / topo->topo.cart->dims[i];
@@ -212,7 +212,7 @@ int smpi_mpi_cart_coords(MPI_Comm comm, int rank, int maxdims, int coords[]) {
 }
 
 int smpi_mpi_cart_get(MPI_Comm comm, int maxdims, int* dims, int* periods, int* coords) {
 }
 
 int smpi_mpi_cart_get(MPI_Comm comm, int maxdims, int* dims, int* periods, int* coords) {
-  MPI_Topology topo = smpi_comm_topo(comm);
+  MPI_Topology topo = comm->topo();
   int ndims=topo->topo.cart->ndims < maxdims ? topo->topo.cart->ndims : maxdims;
   for(int i = 0 ; i < ndims ; i++) {
     dims[i] = topo->topo.cart->dims[i];
   int ndims=topo->topo.cart->ndims < maxdims ? topo->topo.cart->ndims : maxdims;
   for(int i = 0 ; i < ndims ; i++) {
     dims[i] = topo->topo.cart->dims[i];
@@ -223,7 +223,7 @@ int smpi_mpi_cart_get(MPI_Comm comm, int maxdims, int* dims, int* periods, int*
 }
 
 int smpi_mpi_cart_rank(MPI_Comm comm, int* coords, int* rank) {
 }
 
 int smpi_mpi_cart_rank(MPI_Comm comm, int* coords, int* rank) {
-  MPI_Topology topo = smpi_comm_topo(comm);
+  MPI_Topology topo = comm->topo();
   int ndims = topo->topo.cart->ndims;
   int coord;
   *rank = 0;
   int ndims = topo->topo.cart->ndims;
   int coord;
   *rank = 0;
@@ -261,7 +261,7 @@ int smpi_mpi_cart_rank(MPI_Comm comm, int* coords, int* rank) {
 }
 
 int smpi_mpi_cart_shift(MPI_Comm comm, int direction, int disp, int *rank_source, int *rank_dest) {
 }
 
 int smpi_mpi_cart_shift(MPI_Comm comm, int direction, int disp, int *rank_source, int *rank_dest) {
-  MPI_Topology topo = smpi_comm_topo(comm);
+  MPI_Topology topo = comm->topo();
   int position[topo->topo.cart->ndims];
 
   if(topo->topo.cart->ndims == 0) {
   int position[topo->topo.cart->ndims];
 
   if(topo->topo.cart->ndims == 0) {
@@ -271,7 +271,7 @@ int smpi_mpi_cart_shift(MPI_Comm comm, int direction, int disp, int *rank_source
     return MPI_ERR_DIMS;
   }
 
     return MPI_ERR_DIMS;
   }
 
-  smpi_mpi_cart_coords(comm, smpi_comm_rank(comm), topo->topo.cart->ndims, position);
+  smpi_mpi_cart_coords(comm, comm->rank(), topo->topo.cart->ndims, position);
   position[direction] += disp;
 
   if(position[direction] < 0 ||
   position[direction] += disp;
 
   if(position[direction] < 0 ||
@@ -302,7 +302,7 @@ int smpi_mpi_cart_shift(MPI_Comm comm, int direction, int disp, int *rank_source
 }
 
 int smpi_mpi_cartdim_get(MPI_Comm comm, int *ndims) {
 }
 
 int smpi_mpi_cartdim_get(MPI_Comm comm, int *ndims) {
-  MPI_Topology topo = smpi_comm_topo(comm);
+  MPI_Topology topo = comm->topo();
 
   *ndims = topo->topo.cart->ndims;
   return MPI_SUCCESS;
 
   *ndims = topo->topo.cart->ndims;
   return MPI_SUCCESS;
index 13409f2..f4f901e 100644 (file)
@@ -207,7 +207,6 @@ set(SMPI_SRC
   src/smpi/colls/smpi_openmpi_selector.cpp
   src/smpi/colls/smpi_mvapich2_selector.cpp
   src/smpi/instr_smpi.cpp
   src/smpi/colls/smpi_openmpi_selector.cpp
   src/smpi/colls/smpi_mvapich2_selector.cpp
   src/smpi/instr_smpi.cpp
-  src/smpi/forward.hpp
   src/smpi/smpi_base.cpp
   src/smpi/smpi_bench.cpp
   src/smpi/smpi_memory.cpp
   src/smpi/smpi_base.cpp
   src/smpi/smpi_bench.cpp
   src/smpi/smpi_memory.cpp
@@ -671,6 +670,7 @@ set(headers_to_install
   include/smpi/smpi_main.h
   include/smpi/smpi_extended_traces.h
   include/smpi/smpi_extended_traces_fortran.h
   include/smpi/smpi_main.h
   include/smpi/smpi_extended_traces.h
   include/smpi/smpi_extended_traces_fortran.h
+  include/smpi/forward.hpp
   include/surf/surf_routing.h
   include/xbt.h
   include/xbt/RngStream.h
   include/surf/surf_routing.h
   include/xbt.h
   include/xbt/RngStream.h