Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
started alltoallv(). Still some bugs.
authorgenaud <genaud@48e7efb5-ca39-0410-a469-dd3cf9ba447f>
Fri, 31 Jul 2009 17:57:42 +0000 (17:57 +0000)
committergenaud <genaud@48e7efb5-ca39-0410-a469-dd3cf9ba447f>
Fri, 31 Jul 2009 17:57:42 +0000 (17:57 +0000)
git-svn-id: svn+ssh://scm.gforge.inria.fr/svn/simgrid/simgrid/trunk@6538 48e7efb5-ca39-0410-a469-dd3cf9ba447f

ChangeLog
examples/smpi/alltoall2.c
examples/smpi/alltoall_basic.c [new file with mode: 0644]
include/smpi/smpi.h
src/smpi/smpi_coll.c
src/smpi/smpi_coll_private.h
src/smpi/smpi_mpi.c
src/smpi/smpi_mpi_dt.c

index a99f060..0b5b80a 100644 (file)
--- a/ChangeLog
+++ b/ChangeLog
@@ -7,13 +7,14 @@ SimGrid (3.3.2-svn) unstable; urgency=low
 
  SMPI:
   * Implement some more MPI primitives: 
 
  SMPI:
   * Implement some more MPI primitives: 
-    MPI_Bcast, MPI_Waitany, MPI_Waitall, MPI_Reduce, MPI_Allreduce, MPI_Sendrecv, MPI_Alltoall
+    MPI_Bcast, MPI_Waitany, MPI_Waitall, MPI_Reduce, MPI_Allreduce, MPI_Scatter, MPI_Sendrecv, MPI_Alltoall
     -implementation: Bcast: flat or 2-ary tree (default), 
                      Barrier: 4-ary tree,
                      Reduce: flat tree
                      Allreduce: Reduce then Bcast
     -implementation: Bcast: flat or 2-ary tree (default), 
                      Barrier: 4-ary tree,
                      Reduce: flat tree
                      Allreduce: Reduce then Bcast
-                     Alltoall: basic_linear if data per proc < 3Kb, otherwise pairwise. 
-                               Not yet implemented: bruck for data per proc < 200b and comm size > 12
+                     Alltoall: "basic_linear" if data per proc < 3Kb, "otherwise pairwise". 
+                               Not yet implemented: "Bruck" for data per proc < 200b and comm size > 12
+                     Scatter: flat tree
   * Add support for optimized collectives (Bcast is now binomial by default)
   * Port smpirun and smpicc to OS X
 
   * Add support for optimized collectives (Bcast is now binomial by default)
   * Port smpirun and smpicc to OS X
 
index ae47160..0b1c918 100644 (file)
 
 #define MAXLEN 10000
  
 
 #define MAXLEN 10000
  
-void main()
+int main( int argc, char *argv[] )
 {
 {
-   int out[1000000],in[1000000],i,j,k;
+#define N 1000000
+   int *out, *in,i,j,k;
    int myself,tasks;
  
    int myself,tasks;
  
-   MPI_Init(0,0);
+   out = malloc(N*sizeof(int));
+   in  = malloc(N*sizeof(int));
+   if ((out==NULL) || (in==NULL)) {
+           printf("Error: cannot allocate N bytes for in or out arrays\n");
+           exit(1);
+   }
+   MPI_Init( &argc,&argv );
    MPI_Comm_rank(MPI_COMM_WORLD,&myself);
    MPI_Comm_size(MPI_COMM_WORLD,&tasks);
    for(j=1;j<=MAXLEN;j*=10)  {
    MPI_Comm_rank(MPI_COMM_WORLD,&myself);
    MPI_Comm_size(MPI_COMM_WORLD,&tasks);
    for(j=1;j<=MAXLEN;j*=10)  {
-      for(i=0;i<j*tasks;i++)  out[i] = myself;
-      MPI_Alltoall(out,j,MPI_INT,in,j,MPI_INT,MPI_COMM_WORLD);
-      for(i=0;i<tasks;i++)  {
-         for(k=0;k<j;k++) {
-            if(in[k+i*j] != i) {  printf("bad answer (%d) at index %d of %d (should be %d)\n",in[k+i*j],k+i*j,j*tasks,i); break; }
-         }
-      }
+           if ( 0 == myself ) {
+                   printf("* calling MPI_Alltoall with buffers of %d ints\n",j);
+           }
+           for(i=0;i<j*tasks;i++)  
+                   out[i] = myself;
+
+           MPI_Alltoall(out,j,MPI_INT,in,j,MPI_INT,MPI_COMM_WORLD);
+
+           for(i=0;i<tasks;i++)  {
+                   for(k=0;k<j;k++) {
+                           if(in[k+i*j] != i) {  
+                                   printf("<%d> bad answer (%d) at index %d of %d (should be %d)\n",myself,in[k+i*j],k+i*j,j*tasks,i); 
+                                   break; 
+                           }
+                   }
+           }
    }
    MPI_Barrier(MPI_COMM_WORLD);
    if(myself==0)  printf("TEST COMPLETE\n");
    }
    MPI_Barrier(MPI_COMM_WORLD);
    if(myself==0)  printf("TEST COMPLETE\n");
diff --git a/examples/smpi/alltoall_basic.c b/examples/smpi/alltoall_basic.c
new file mode 100644 (file)
index 0000000..12bc44d
--- /dev/null
@@ -0,0 +1,56 @@
+#include "mpi.h"
+#include <stdlib.h>
+#include <stdio.h>
+#include <string.h>
+#include <errno.h>
+
+#ifndef EXIT_SUCCESS
+#define EXIT_SUCCESS 0
+#define EXIT_FAILURE 1
+#endif
+
+int main( int argc, char *argv[] )
+{
+    int rank, size;
+    int i;
+    int *sb;
+    int *rb;
+    int status, gstatus;
+
+    MPI_Init(&argc,&argv);
+    MPI_Comm_rank(MPI_COMM_WORLD,&rank);
+    MPI_Comm_size(MPI_COMM_WORLD,&size);
+    
+    sb = (int *)malloc(size*sizeof(int));
+    if ( !sb ) {
+        perror( "can't allocate send buffer" );fflush(stderr);
+        MPI_Abort(MPI_COMM_WORLD,EXIT_FAILURE);
+    }
+    rb = (int *)malloc(size*sizeof(int));
+    if ( !rb ) {
+        perror( "can't allocate recv buffer");fflush(stderr);
+        free(sb);
+        MPI_Abort(MPI_COMM_WORLD, EXIT_FAILURE);
+    }
+    for ( i=0 ; i < size ; ++i ) {
+        sb[i] = rank + 1;
+        rb[i] = 0;
+    }
+    status = MPI_Alltoall(sb, 1, MPI_INT, rb, 1, MPI_INT, MPI_COMM_WORLD);
+
+    printf("[%d] rcvbuf=[",rank);
+    for (i=0;i<size;i++) 
+            printf("%d ",rb[i]);
+    printf("]\n");
+
+
+    if (rank == 0) {
+        if (gstatus != 0) {
+            printf("all_to_all returned %d\n",gstatus);fflush(stdout);
+        }
+    }
+    free(sb);
+    free(rb);
+    MPI_Finalize();
+    return(EXIT_SUCCESS);
+}
index 2f0df2d..928d16c 100644 (file)
@@ -109,10 +109,12 @@ SG_BEGIN_DECL()
 #define MPI_Waitall(a, b, c) SMPI_MPI_Waitall(a, b, c)
 #define MPI_Waitany(a, b, c, d) SMPI_MPI_Waitany(a, b, c, d)
 #define MPI_Wtime() SMPI_MPI_Wtime()
 #define MPI_Waitall(a, b, c) SMPI_MPI_Waitall(a, b, c)
 #define MPI_Waitany(a, b, c, d) SMPI_MPI_Waitany(a, b, c, d)
 #define MPI_Wtime() SMPI_MPI_Wtime()
-#define MPI_Reduce( a, b, c, d, e, f, g) SMPI_MPI_Reduce( a, b, c, d, e, f, g)
-#define MPI_Allreduce( a, b, c, d, e, f) SMPI_MPI_Allreduce( a, b, c, d, e, f)
-#define MPI_Scatter( a, b, c, d, e, f, g, h )  SMPI_MPI_Scatter( a, b, c, d, e, f, g, h)
-#define MPI_Alltoall( a, b, c, d, e, f, g )  SMPI_MPI_Alltoall( a, b, c, d, e, f, g)
+#define MPI_Reduce(a, b, c, d, e, f, g) SMPI_MPI_Reduce(a, b, c, d, e, f, g)
+#define MPI_Allreduce(a, b, c, d, e, f) SMPI_MPI_Allreduce(a, b, c, d, e, f)
+#define MPI_Scatter(a, b, c, d, e, f, g, h )  SMPI_MPI_Scatter(a, b, c, d, e, f, g, h)
+#define MPI_Alltoall(a, b, c, d, e, f, g )  SMPI_MPI_Alltoall(a, b, c, d, e, f, g)
+#define MPI_Alltoallv(a, b, c, d, e, f, g, h, i)  SMPI_MPI_Alltoallv(a, b, c, d, e, f, g, h, i)
+
 
 // SMPI Functions
 XBT_PUBLIC(int) SMPI_MPI_Init(int *argc, char ***argv);
 
 // SMPI Functions
 XBT_PUBLIC(int) SMPI_MPI_Init(int *argc, char ***argv);
@@ -165,6 +167,10 @@ XBT_PUBLIC(int) SMPI_MPI_Scatter(void *sendbuf, int sendcount, MPI_Datatype data
 
 XBT_PUBLIC(int) SMPI_MPI_Alltoall(void *sendbuf, int sendcount, MPI_Datatype datatype,
                                     void *recvbuf, int recvcount, MPI_Datatype recvtype, MPI_Comm comm);
 
 XBT_PUBLIC(int) SMPI_MPI_Alltoall(void *sendbuf, int sendcount, MPI_Datatype datatype,
                                     void *recvbuf, int recvcount, MPI_Datatype recvtype, MPI_Comm comm);
+XBT_PUBLIC(int) SMPI_MPI_Alltoallv(void *sendbuf, int *scounts, int *sdisps, MPI_Datatype datatype, 
+                          void *recvbuf, int *rcounts, int *rdisps, MPI_Datatype recvtype,
+                            MPI_Comm comm);
+
 
 // smpi functions
 XBT_IMPORT_NO_EXPORT(int) smpi_simulated_main(int argc, char **argv);
 
 // smpi functions
 XBT_IMPORT_NO_EXPORT(int) smpi_simulated_main(int argc, char **argv);
index 6881592..43738f1 100644 (file)
@@ -229,7 +229,7 @@ int tree_antibcast( void *buf, int count, MPI_Datatype datatype, int root,
                                 system_tag + rank, 
                                 comm, &request);
                 if (MPI_SUCCESS != retval) {
                                 system_tag + rank, 
                                 comm, &request);
                 if (MPI_SUCCESS != retval) {
-                        printf("** internal error: smpi_create_request() rank=%d returned retval=%d, %s:%d\n",
+                        ERROR4("** internal error: smpi_create_request() rank=%d returned retval=%d, %s:%d\n",
                                         rank,retval,__FILE__,__LINE__);
                 }
                 smpi_mpi_isend(request);
                                         rank,retval,__FILE__,__LINE__);
                 }
                 smpi_mpi_isend(request);
@@ -389,7 +389,7 @@ int copy_dt( void *sbuf, int scount, const MPI_Datatype sdtype,
 }
 
 /**
 }
 
 /**
- *
+ * Alltoall basic_linear
  **/
 int smpi_coll_tuned_alltoall_basic_linear(void *sbuf, int scount, MPI_Datatype sdtype,
                    void* rbuf, int rcount, MPI_Datatype rdtype, MPI_Comm comm)
  **/
 int smpi_coll_tuned_alltoall_basic_linear(void *sbuf, int scount, MPI_Datatype sdtype,
                    void* rbuf, int rcount, MPI_Datatype rdtype, MPI_Comm comm)
@@ -640,6 +640,157 @@ err_hndl:
          return -1; /* FIXME: to be changed*/
 }
 
          return -1; /* FIXME: to be changed*/
 }
 
+static void print_buffer_int(void *buf, int len, char *msg, int rank);
+static void print_buffer_int(void *buf, int len, char *msg, int rank)
+{
+  int tmp, *v;
+  fprintf(stderr,"**<%d> %s (#%d): ", rank, msg,len);
+  for (tmp = 0; tmp < len; tmp++) {
+    v = buf;
+    fprintf(stderr,"[%d (%p)]", v[tmp],v+tmp);
+  }
+  fprintf(stderr,"\n");
+  free(msg);
+}
+
+
+
+/**
+ * alltoallv basic 
+ **/
+
+int smpi_coll_basic_alltoallv(void *sbuf, int *scounts, int *sdisps, MPI_Datatype sdtype, 
+                              void *rbuf, int *rcounts, int *rdisps, MPI_Datatype rdtype,
+                              MPI_Comm comm) {
+
+        int i;
+        int system_alltoallv_tag = 889;
+        int rank;
+        int size = comm->size;
+        int err;
+        char *psnd;
+        char *prcv;
+        //int nreq = 0;
+        int rreq = 0;
+        int sreq = 0;
+        MPI_Aint lb;
+        MPI_Aint sndextent;
+        MPI_Aint rcvextent;
+        MPI_Request *reqs;
+
+        /* Initialize. */
+        rank = smpi_mpi_comm_rank(comm);
+        DEBUG1("<%d> algorithm basic_alltoallv() called.",rank);
+
+        err = smpi_mpi_type_get_extent(sdtype, &lb, &sndextent);
+        err = smpi_mpi_type_get_extent(rdtype, &lb, &rcvextent);
+        DEBUG3("<%d> sizeof(sndttype)=%d,sizeof(rcvtype)=%d",rank,sndextent,rcvextent);
+
+        psnd = (char *)sbuf;
+        print_buffer_int(psnd,size*size,xbt_strdup("sbuff"),rank);
+
+        /* copy the local sbuf to rbuf when it's me */
+        psnd = ((char *) sbuf) + (sdisps[rank] * sndextent);
+        prcv = ((char *) rbuf) + (rdisps[rank] * rcvextent);
+
+        if (0 != scounts[rank]) {
+                err = copy_dt( psnd, scounts[rank], sdtype, prcv, rcounts[rank], rdtype );
+                print_buffer_int(psnd,scounts[rank],strdup("copy_dt"),rank);
+                if (MPI_SUCCESS != err) {
+                        return err;
+                }
+        }
+
+        /* If only one process, we're done. */
+        if (1 == size) {
+                return MPI_SUCCESS;
+        }
+
+        /* Initiate all send/recv to/from others. */
+        reqs =  xbt_malloc(2*(size-1) * sizeof(smpi_mpi_request_t));
+
+
+        /* Create all receives that will be posted first */
+        for (i = 0; i < size; ++i) {
+                if (i == rank || 0 == rcounts[i]) {
+                        DEBUG3("<%d> skip req creation i=%d,rcounts[i]=%d",rank,i, rcounts[i]);
+                        continue;
+                }
+                prcv = ((char *) rbuf) + (rdisps[i] * rcvextent);
+
+                err = smpi_create_request( prcv, rcounts[i], rdtype,
+                                i, rank,
+                                system_alltoallv_tag,
+                                comm, &(reqs[rreq]));
+                if (MPI_SUCCESS != err) {
+                        DEBUG2("<%d> failed to create request for rank %d",rank,i);
+                        for (i=0;i< rreq;i++) 
+                                xbt_mallocator_release(smpi_global->request_mallocator, reqs[i]);
+                        return err;
+                }
+                rreq++;
+        }
+        DEBUG2("<%d> %d irecv reqs created",rank,rreq);
+        /* Now create all sends  */
+        for (i = 0; i < size; ++i) {
+                if (i == rank || 0 == scounts[i]) {
+                        DEBUG3("<%d> skip req creation i=%d,scounts[i]=%d",rank,i, scounts[i]);
+                        continue;
+                }
+                psnd = ((char *) sbuf) + (sdisps[i] * sndextent);
+
+                fprintf(stderr,"<%d> send %d elems to <%d>\n",rank,scounts[i],i);
+                print_buffer_int(psnd,scounts[i],xbt_strdup("sbuff part"),rank);
+                err = smpi_create_request (psnd, scounts[i], sdtype,
+                                rank, i,
+                                system_alltoallv_tag, 
+                                comm, &(reqs[rreq+sreq]));
+                if (MPI_SUCCESS != err) {
+                        DEBUG2("<%d> failed to create request for rank %d\n",rank,i);
+                        for (i=0;i< rreq+sreq;i++) 
+                                xbt_mallocator_release(smpi_global->request_mallocator, reqs[i]);
+                        return err;
+                }
+                sreq++;
+        }
+        DEBUG2("<%d> %d isend reqs created",rank,sreq);
+
+        /* Start your engines.  This will never return an error. */
+        for ( i=0; i< rreq; i++ ) {
+                DEBUG3("<%d> issued irecv request reqs[%d]=%p",rank,i,reqs[i]);
+                smpi_mpi_irecv( reqs[i] );
+        }
+        for ( i=rreq; i<sreq; i++ ) {
+                DEBUG3("<%d> issued isend request reqs[%d]=%p",rank,i,reqs[i]);
+                smpi_mpi_isend( reqs[i] );
+        }
+
+
+        /* Wait for them all.  If there's an error, note that we don't
+         * care what the error was -- just that there *was* an error.  The
+         * PML will finish all requests, even if one or more of them fail.
+         * i.e., by the end of this call, all the requests are free-able.
+         * So free them anyway -- even if there was an error, and return
+         * the error after we free everything. */
+
+        DEBUG2("<%d> wait for %d requests",rank,rreq+sreq);
+        // waitall is buggy: use a loop instead for the moment
+        // err = smpi_mpi_waitall(nreq, reqs, MPI_STATUS_IGNORE);
+        for (i=0;i< rreq+sreq;i++) {
+                err = smpi_mpi_wait( reqs[i], MPI_STATUS_IGNORE);
+        }
+
+        /* Free the reqs */
+        /* nreq might be < 2*(size-1) since some request creations are skipped */
+        for (i=0;i< rreq+sreq;i++) {
+                xbt_mallocator_release(smpi_global->request_mallocator, reqs[i]);
+        }
+        xbt_free( reqs );
+        return err;
+}
+
+
+
 
 /**
  * -----------------------------------------------------------------------------------------------------
 
 /**
  * -----------------------------------------------------------------------------------------------------
index b4b1e9d..31a0bd7 100644 (file)
@@ -22,3 +22,8 @@ int smpi_coll_tuned_alltoall_pairwise (void *sendbuf, int sendcount, MPI_Datatyp
 int smpi_coll_tuned_alltoall_basic_linear(void *sbuf, int scount, MPI_Datatype sdtype,
                                 void* rbuf, int rcount, MPI_Datatype rdtype, MPI_Comm comm);
 
 int smpi_coll_tuned_alltoall_basic_linear(void *sbuf, int scount, MPI_Datatype sdtype,
                                 void* rbuf, int rcount, MPI_Datatype rdtype, MPI_Comm comm);
 
+int smpi_coll_basic_alltoallv(void *sendbuf, int *scounts, int *sdisps, MPI_Datatype datatype, 
+                                 void *recvbuf, int *rcounts, int *rdisps, MPI_Datatype recvtype,
+                                   MPI_Comm comm);
+
+
index fdd8a4f..53799a0 100644 (file)
@@ -1,3 +1,14 @@
+/* $Id: $tag */
+
+/* smpi_mpi.c -- 
+ *
+ * Eventually will contain the user level MPI primitives and its corresponding 
+ * internal wrapper. The implementations of these primitives should go to specific
+ * files. For example, a SMPI_MPI_Bcast() in this file, should call the wrapper 
+ * smpi_mpi_bcast(), which decides which implementation to call. Currently, it
+ * calls nary_tree_bcast() in smpi_coll.c.  (Stéphane Genaud).
+ * */
+
 
 
 #include "private.h"
 
 
 #include "private.h"
@@ -457,7 +468,7 @@ int retval = MPI_SUCCESS;
 /**
  * MPI_Allreduce
  *
 /**
  * MPI_Allreduce
  *
- * Same as MPI_REDUCE except that the result appears in the receive buffer of all the group members.
+ * Same as MPI_Reduce except that the result appears in the receive buffer of all the group members.
  **/
 int SMPI_MPI_Allreduce( void *sendbuf, void *recvbuf, int count, MPI_Datatype datatype,
                    MPI_Op op, MPI_Comm comm )
  **/
 int SMPI_MPI_Allreduce( void *sendbuf, void *recvbuf, int count, MPI_Datatype datatype,
                    MPI_Op op, MPI_Comm comm )
@@ -555,8 +566,8 @@ int SMPI_MPI_Scatter(void *sendbuf, int sendcount, MPI_Datatype datatype,
  * ompi/mca/coll/tuned/coll_tuned_module.c
  **/
 int SMPI_MPI_Alltoall(void *sendbuf, int sendcount, MPI_Datatype datatype, 
  * ompi/mca/coll/tuned/coll_tuned_module.c
  **/
 int SMPI_MPI_Alltoall(void *sendbuf, int sendcount, MPI_Datatype datatype, 
-                        void *recvbuf, int recvcount, MPI_Datatype recvtype,
-                          MPI_Comm comm)
+                         void *recvbuf, int recvcount, MPI_Datatype recvtype,
+                           MPI_Comm comm)
 {
   int retval = MPI_SUCCESS;
   int block_dsize;
 {
   int retval = MPI_SUCCESS;
   int block_dsize;
@@ -586,6 +597,29 @@ int SMPI_MPI_Alltoall(void *sendbuf, int sendcount, MPI_Datatype datatype,
   return retval;
 }
 
   return retval;
 }
 
+/**
+ * MPI_Alltoallv user entry point
+ * 
+ * As in OpenMPI, alltoallv is not optimized
+ * ompi/mca/coll/basic/coll_basic_alltoallv.c 
+ **/
+int SMPI_MPI_Alltoallv(void *sendbuf, int *scounts, int *sdisps, MPI_Datatype datatype, 
+                          void *recvbuf, int *rcounts, int *rdisps, MPI_Datatype recvtype,
+                            MPI_Comm comm)
+{
+  int retval = MPI_SUCCESS;
+  int rank;
+
+  rank = smpi_mpi_comm_rank(comm);
+  DEBUG1("<%d> basic alltoallv() called.",rank);
+
+  retval = smpi_coll_basic_alltoallv(sendbuf, scounts, sdisps, datatype, 
+                                     recvbuf, rcounts, rdisps, recvtype,
+                                     comm); 
+
+  return retval;
+}
+
 
 
 
 
 
 
index 7352a09..a196e36 100644 (file)
@@ -47,6 +47,16 @@ int smpi_mpi_type_get_extent(MPI_Datatype datatype, MPI_Aint *lb, MPI_Aint *exte
         *extent =  datatype->ub - datatype->lb;
         return( MPI_SUCCESS );
 }
         *extent =  datatype->ub - datatype->lb;
         return( MPI_SUCCESS );
 }
+
+
+/**
+ * query extent and lower bound of the type 
+ **/
+int SMPI_MPI_Type_get_extent( MPI_Datatype datatype, int *lb, int *extent) 
+{
+        return( smpi_mpi_type_get_extent( datatype, lb, extent));
+}
+
 /**
  * query the size of the type
  **/
 /**
  * query the size of the type
  **/
@@ -70,15 +80,6 @@ int SMPI_MPI_Type_size(MPI_Datatype datatype, size_t * size)
 }
 
 
 }
 
 
-
-/**
- * query extent and lower bound of the type 
- **/
-int SMPI_MPI_Type_get_extent( MPI_Datatype datatype, int *lb, int *extent) 
-{
-        return( smpi_mpi_type_get_extent( datatype, lb, extent));
-}
-
 /* Deprecated Functions. 
  * The MPI-2 standard deprecated a number of routines because MPI-2 provides better versions. 
  * This routine is one of those that was deprecated. The routine may continue to be used, but 
 /* Deprecated Functions. 
  * The MPI-2 standard deprecated a number of routines because MPI-2 provides better versions. 
  * This routine is one of those that was deprecated. The routine may continue to be used, but