Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
set default error handler to MPI_ERRORS_ARE_FATAL, as in a proper MPI implementation
[simgrid.git] / src / smpi / mpi / smpi_win.cpp
index f2782d1..953584f 100644 (file)
@@ -1,20 +1,22 @@
-/* Copyright (c) 2007-2017. The SimGrid Team. All rights reserved.          */
+/* Copyright (c) 2007-2019. 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. */
 
-#include "private.h"
+#include "smpi_win.hpp"
+
+#include "private.hpp"
 #include "smpi_coll.hpp"
 #include "smpi_comm.hpp"
 #include "smpi_datatype.hpp"
 #include "smpi_info.hpp"
 #include "smpi_keyvals.hpp"
-#include "smpi_process.hpp"
 #include "smpi_request.hpp"
-#include "smpi_win.hpp"
+#include "src/smpi/include/smpi_actor.hpp"
 
 XBT_LOG_NEW_DEFAULT_SUBCATEGORY(smpi_rma, smpi, "Logging specific to SMPI (RMA operations)");
 
+
 namespace simgrid{
 namespace smpi{
 std::unordered_map<int, smpi_key_elem> Win::keyvals_;
@@ -22,38 +24,39 @@ int Win::keyval_id_=0;
 
 Win::Win(void *base, MPI_Aint size, int disp_unit, MPI_Info info, MPI_Comm comm, int allocated, int dynamic): base_(base), size_(size), disp_unit_(disp_unit), assert_(0), info_(info), comm_(comm), allocated_(allocated), dynamic_(dynamic){
   int comm_size = comm->size();
-  rank_      = comm->rank();
+  rank_         = comm->rank();
   XBT_DEBUG("Creating window");
   if(info!=MPI_INFO_NULL)
     info->ref();
-  name_ = nullptr;
-  opened_ = 0;
-  group_ = MPI_GROUP_NULL;
-  requests_ = new std::vector<MPI_Request>();
-  mut_=xbt_mutex_init();
-  lock_mut_=xbt_mutex_init();
-  atomic_mut_=xbt_mutex_init();
-  connected_wins_ = new MPI_Win[comm_size];
+  name_                  = nullptr;
+  opened_                = 0;
+  group_                 = MPI_GROUP_NULL;
+  requests_              = new std::vector<MPI_Request>();
+  mut_                   = s4u::Mutex::create();
+  lock_mut_              = s4u::Mutex::create();
+  atomic_mut_            = s4u::Mutex::create();
+  connected_wins_        = new MPI_Win[comm_size];
   connected_wins_[rank_] = this;
-  count_ = 0;
+  count_                 = 0;
   if(rank_==0){
-    bar_ = MSG_barrier_init(comm_size);
+    bar_ = new s4u::Barrier(comm_size);
   }
   mode_=0;
-
+  errhandler_=MPI_ERRORS_ARE_FATAL;
   comm->add_rma_win(this);
+  comm->ref();
 
   Colls::allgather(&(connected_wins_[rank_]), sizeof(MPI_Win), MPI_BYTE, connected_wins_, sizeof(MPI_Win),
                          MPI_BYTE, comm);
 
-  Colls::bcast(&(bar_), sizeof(msg_bar_t), MPI_BYTE, 0, comm);
+  Colls::bcast(&(bar_), sizeof(s4u::Barrier*), MPI_BYTE, 0, comm);
 
   Colls::barrier(comm);
 }
 
 Win::~Win(){
   //As per the standard, perform a barrier to ensure every async comm is finished
-  MSG_barrier_wait(bar_);
+  bar_->wait();
 
   int finished = finish_comms();
   XBT_DEBUG("Win destructor - Finished %d RMA calls", finished);
@@ -70,12 +73,10 @@ Win::~Win(){
   comm_->remove_rma_win(this);
 
   Colls::barrier(comm_);
-  int rank=comm_->rank();
-  if(rank == 0)
-    MSG_barrier_destroy(bar_);
-  xbt_mutex_destroy(mut_);
-  xbt_mutex_destroy(lock_mut_);
-  xbt_mutex_destroy(atomic_mut_);
+  Comm::unref(comm_);
+  
+  if (rank_ == 0)
+    delete bar_;
 
   if(allocated_ !=0)
     xbt_free(base_);
@@ -83,7 +84,8 @@ Win::~Win(){
   cleanup_attr<Win>();
 }
 
-int Win::attach (void *base, MPI_Aint size){
+int Win::attach(void* /*base*/, MPI_Aint size)
+{
   if (not(base_ == MPI_BOTTOM || base_ == 0))
     return MPI_ERR_ARG;
   base_=0;//actually the address will be given in the RMA calls, as being the disp.
@@ -91,7 +93,8 @@ int Win::attach (void *base, MPI_Aint size){
   return MPI_SUCCESS;
 }
 
-int Win::detach (void *base){
+int Win::detach(const void* /*base*/)
+{
   base_=MPI_BOTTOM;
   size_=-1;
   return MPI_SUCCESS;
@@ -148,7 +151,7 @@ void Win::set_info(MPI_Info info){
   info_=info;
 }
 
-void Win::set_name(char* name){
+void Win::set_name(const char* name){
   name_ = xbt_strdup(name);
 }
 
@@ -159,8 +162,8 @@ int Win::fence(int assert)
     opened_=1;
   if (assert != MPI_MODE_NOPRECEDE) {
     // This is not the first fence => finalize what came before
-    MSG_barrier_wait(bar_);
-    xbt_mutex_acquire(mut_);
+    bar_->wait();
+    mut_->lock();
     // This (simulated) mutex ensures that no process pushes to the vector of requests during the waitall.
     // Without this, the vector could get redimensionned when another process pushes.
     // This would result in the array used by Request::waitall() to be invalidated.
@@ -173,20 +176,20 @@ int Win::fence(int assert)
       Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
     }
     count_=0;
-    xbt_mutex_release(mut_);
+    mut_->unlock();
   }
 
   if(assert==MPI_MODE_NOSUCCEED)//there should be no ops after this one, tell we are closed.
     opened_=0;
   assert_ = assert;
 
-  MSG_barrier_wait(bar_);
+  bar_->wait();
   XBT_DEBUG("Leaving fence");
 
   return MPI_SUCCESS;
 }
 
-int Win::put( void *origin_addr, int origin_count, MPI_Datatype origin_datatype, int target_rank,
+int Win::put(const void *origin_addr, int origin_count, MPI_Datatype origin_datatype, int target_rank,
               MPI_Aint target_disp, int target_count, MPI_Datatype target_datatype, MPI_Request* request)
 {
   //get receiver pointer
@@ -195,7 +198,7 @@ int Win::put( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
   if(opened_==0){//check that post/start has been done
     // no fence or start .. lock ok ?
     int locked=0;
-    for(auto it : recv_win->lockers_)
+    for (auto const& it : recv_win->lockers_)
       if (it == comm_->rank())
         locked = 1;
     if(locked != 1)
@@ -206,16 +209,19 @@ int Win::put( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
     return MPI_ERR_ARG;
 
   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 != comm_->rank()){
-    //prepare send_request
-    MPI_Request sreq = Request::rma_send_init(origin_addr, origin_count, origin_datatype, smpi_process()->index(),
-        comm_->group()->index(target_rank), SMPI_RMA_TAG+1, comm_, MPI_OP_NULL);
+  if (target_rank != comm_->rank()) { // This is not for myself, so we need to send messages
+    XBT_DEBUG("Entering MPI_Put to remote rank %d", target_rank);
+    // prepare send_request
+    MPI_Request sreq =
+        // TODO cheinrich Check for rank / pid conversion
+        Request::rma_send_init(origin_addr, origin_count, origin_datatype, comm_->rank(), target_rank, SMPI_RMA_TAG + 1,
+                               comm_, MPI_OP_NULL);
 
     //prepare receiver request
-    MPI_Request rreq = Request::rma_recv_init(recv_addr, target_count, target_datatype, smpi_process()->index(),
-        comm_->group()->index(target_rank), SMPI_RMA_TAG+1, recv_win->comm_, MPI_OP_NULL);
+    // TODO cheinrich Check for rank / pid conversion
+    MPI_Request rreq = Request::rma_recv_init(recv_addr, target_count, target_datatype, recv_win->comm_->rank(),
+                                              target_rank, SMPI_RMA_TAG + 1, recv_win->comm_, MPI_OP_NULL);
 
     //start send
     sreq->start();
@@ -223,18 +229,19 @@ int Win::put( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
     if(request!=nullptr){
       *request=sreq;
     }else{
-      xbt_mutex_acquire(mut_);
+      mut_->lock();
       requests_->push_back(sreq);
-      xbt_mutex_release(mut_);
+      mut_->unlock();
     }
 
     //push request to receiver's win
-    xbt_mutex_acquire(recv_win->mut_);
+    recv_win->mut_->lock();
     recv_win->requests_->push_back(rreq);
     rreq->start();
-    xbt_mutex_release(recv_win->mut_);
+    recv_win->mut_->unlock();
 
   }else{
+    XBT_DEBUG("Entering MPI_Put from myself to myself, rank %d", target_rank);
     Datatype::copy(origin_addr, origin_count, origin_datatype, recv_addr, target_count, target_datatype);
     if(request!=nullptr)
       *request = MPI_REQUEST_NULL;
@@ -252,7 +259,7 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
   if(opened_==0){//check that post/start has been done
     // no fence or start .. lock ok ?
     int locked=0;
-    for(auto it : send_win->lockers_)
+    for (auto const& it : send_win->lockers_)
       if (it == comm_->rank())
         locked = 1;
     if(locked != 1)
@@ -267,21 +274,21 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
 
   if(target_rank != comm_->rank()){
     //prepare send_request
-    MPI_Request sreq = Request::rma_send_init(send_addr, target_count, target_datatype,
-        comm_->group()->index(target_rank), smpi_process()->index(), SMPI_RMA_TAG+2, send_win->comm_,
-        MPI_OP_NULL);
+    MPI_Request sreq = Request::rma_send_init(send_addr, target_count, target_datatype, target_rank,
+                                              send_win->comm_->rank(), SMPI_RMA_TAG + 2, send_win->comm_, MPI_OP_NULL);
 
     //prepare receiver request
-    MPI_Request rreq = Request::rma_recv_init(origin_addr, origin_count, origin_datatype,
-        comm_->group()->index(target_rank), smpi_process()->index(), SMPI_RMA_TAG+2, comm_,
-        MPI_OP_NULL);
+    MPI_Request rreq = Request::rma_recv_init(
+        origin_addr, origin_count, origin_datatype, target_rank,
+        comm_->rank(), // TODO cheinrich Check here if comm_->rank() and above send_win->comm_->rank() are correct
+        SMPI_RMA_TAG + 2, comm_, MPI_OP_NULL);
 
     //start the send, with another process than us as sender.
     sreq->start();
     //push request to receiver's win
-    xbt_mutex_acquire(send_win->mut_);
+    send_win->mut_->lock();
     send_win->requests_->push_back(sreq);
-    xbt_mutex_release(send_win->mut_);
+    send_win->mut_->unlock();
 
     //start recv
     rreq->start();
@@ -289,9 +296,9 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
     if(request!=nullptr){
       *request=rreq;
     }else{
-      xbt_mutex_acquire(mut_);
+      mut_->lock();
       requests_->push_back(rreq);
-      xbt_mutex_release(mut_);
+      mut_->unlock();
     }
 
   }else{
@@ -304,17 +311,17 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
 }
 
 
-int Win::accumulate( void *origin_addr, int origin_count, MPI_Datatype origin_datatype, int target_rank,
+int Win::accumulate(const void *origin_addr, int origin_count, MPI_Datatype origin_datatype, int target_rank,
               MPI_Aint target_disp, int target_count, MPI_Datatype target_datatype, MPI_Op op, MPI_Request* request)
 {
-
+  XBT_DEBUG("Entering MPI_Win_Accumulate");
   //get receiver pointer
   MPI_Win recv_win = connected_wins_[target_rank];
 
   if(opened_==0){//check that post/start has been done
     // no fence or start .. lock ok ?
     int locked=0;
-    for(auto it : recv_win->lockers_)
+    for (auto const& it : recv_win->lockers_)
       if (it == comm_->rank())
         locked = 1;
     if(locked != 1)
@@ -330,45 +337,46 @@ int Win::accumulate( void *origin_addr, int origin_count, MPI_Datatype origin_da
     //As the tag will be used for ordering of the operations, substract count from it (to avoid collisions with other SMPI tags, SMPI_RMA_TAG is set below all the other ones we use )
     //prepare send_request
 
-    MPI_Request sreq = Request::rma_send_init(origin_addr, origin_count, origin_datatype,
-        smpi_process()->index(), comm_->group()->index(target_rank), SMPI_RMA_TAG-3-count_, comm_, op);
+  MPI_Request sreq = Request::rma_send_init(origin_addr, origin_count, origin_datatype, comm_->rank(), target_rank,
+                                            SMPI_RMA_TAG - 3 - count_, comm_, op);
 
-    //prepare receiver request
-    MPI_Request rreq = Request::rma_recv_init(recv_addr, target_count, target_datatype,
-        smpi_process()->index(), comm_->group()->index(target_rank), SMPI_RMA_TAG-3-count_, recv_win->comm_, op);
+  // prepare receiver request
+  MPI_Request rreq = Request::rma_recv_init(recv_addr, target_count, target_datatype, recv_win->comm_->rank(),
+                                            recv_win->comm_->group()->rank(comm_->group()->actor(target_rank)), SMPI_RMA_TAG - 3 - count_, recv_win->comm_, op);
 
-    count_++;
+  count_++;
 
-    //start send
-    sreq->start();
-    //push request to receiver's win
-    xbt_mutex_acquire(recv_win->mut_);
-    recv_win->requests_->push_back(rreq);
-    rreq->start();
-    xbt_mutex_release(recv_win->mut_);
+  // start send
+  sreq->start();
+  // push request to receiver's win
+  recv_win->mut_->lock();
+  recv_win->requests_->push_back(rreq);
+  rreq->start();
+  recv_win->mut_->unlock();
 
-    if(request!=nullptr){
-      *request=sreq;
-    }else{
-      xbt_mutex_acquire(mut_);
-      requests_->push_back(sreq);
-      xbt_mutex_release(mut_);
-    }
+  if (request != nullptr) {
+    *request = sreq;
+  } else {
+    mut_->lock();
+    requests_->push_back(sreq);
+    mut_->unlock();
+  }
 
+  XBT_DEBUG("Leaving MPI_Win_Accumulate");
   return MPI_SUCCESS;
 }
 
-int Win::get_accumulate( void *origin_addr, int origin_count, MPI_Datatype origin_datatype, void *result_addr,
-              int result_count, MPI_Datatype result_datatype, int target_rank, MPI_Aint target_disp, int target_count,
-              MPI_Datatype target_datatype, MPI_Op op, MPI_Request* request){
-
+int Win::get_accumulate(const void* origin_addr, int origin_count, MPI_Datatype origin_datatype, void* result_addr,
+                        int result_count, MPI_Datatype result_datatype, int target_rank, MPI_Aint target_disp,
+                        int target_count, MPI_Datatype target_datatype, MPI_Op op, MPI_Request*)
+{
   //get sender pointer
   MPI_Win send_win = connected_wins_[target_rank];
 
   if(opened_==0){//check that post/start has been done
     // no fence or start .. lock ok ?
     int locked=0;
-    for(auto it : send_win->lockers_)
+    for (auto const& it : send_win->lockers_)
       if (it == comm_->rank())
         locked = 1;
     if(locked != 1)
@@ -381,7 +389,7 @@ int Win::get_accumulate( void *origin_addr, int origin_count, MPI_Datatype origi
   XBT_DEBUG("Entering MPI_Get_accumulate from %d", target_rank);
   //need to be sure ops are correctly ordered, so finish request here ? slow.
   MPI_Request req;
-  xbt_mutex_acquire(send_win->atomic_mut_);
+  send_win->atomic_mut_->lock();
   get(result_addr, result_count, result_datatype, target_rank,
               target_disp, target_count, target_datatype, &req);
   if (req != MPI_REQUEST_NULL)
@@ -391,12 +399,12 @@ int Win::get_accumulate( void *origin_addr, int origin_count, MPI_Datatype origi
               target_disp, target_count, target_datatype, op, &req);
   if (req != MPI_REQUEST_NULL)
     Request::wait(&req, MPI_STATUS_IGNORE);
-  xbt_mutex_release(send_win->atomic_mut_);
+  send_win->atomic_mut_->unlock();
   return MPI_SUCCESS;
 
 }
 
-int Win::compare_and_swap(void *origin_addr, void *compare_addr,
+int Win::compare_and_swap(const void *origin_addr, void *compare_addr,
         void *result_addr, MPI_Datatype datatype, int target_rank,
         MPI_Aint target_disp){
   //get sender pointer
@@ -405,7 +413,7 @@ int Win::compare_and_swap(void *origin_addr, void *compare_addr,
   if(opened_==0){//check that post/start has been done
     // no fence or start .. lock ok ?
     int locked=0;
-    for(auto it : send_win->lockers_)
+    for (auto const& it : send_win->lockers_)
       if (it == comm_->rank())
         locked = 1;
     if(locked != 1)
@@ -413,8 +421,8 @@ int Win::compare_and_swap(void *origin_addr, void *compare_addr,
   }
 
   XBT_DEBUG("Entering MPI_Compare_and_swap with %d", target_rank);
-  MPI_Request req;
-  xbt_mutex_acquire(send_win->atomic_mut_);
+  MPI_Request req = MPI_REQUEST_NULL;
+  send_win->atomic_mut_->lock();
   get(result_addr, 1, datatype, target_rank,
               target_disp, 1, datatype, &req);
   if (req != MPI_REQUEST_NULL)
@@ -423,61 +431,66 @@ int Win::compare_and_swap(void *origin_addr, void *compare_addr,
     put(origin_addr, 1, datatype, target_rank,
               target_disp, 1, datatype);
   }
-  xbt_mutex_release(send_win->atomic_mut_);
+  send_win->atomic_mut_->unlock();
   return MPI_SUCCESS;
 }
 
-int Win::start(MPI_Group group, int assert){
-    /* From MPI forum advices
-    The call to MPI_WIN_COMPLETE does not return until the put call has completed at the origin; and the target window
-    will be accessed by the put operation only after the call to MPI_WIN_START has matched a call to MPI_WIN_POST by
-    the target process. This still leaves much choice to implementors. The call to MPI_WIN_START can block until the
-    matching call to MPI_WIN_POST occurs at all target processes. One can also have implementations where the call to
-    MPI_WIN_START is nonblocking, but the call to MPI_PUT blocks until the matching call to MPI_WIN_POST occurred; or
-    implementations where the first two calls are nonblocking, but the call to MPI_WIN_COMPLETE blocks until the call
-    to MPI_WIN_POST occurred; or even implementations where all three calls can complete before any target process
-    called MPI_WIN_POST --- the data put must be buffered, in this last case, so as to allow the put to complete at the
-    origin ahead of its completion at the target. However, once the call to MPI_WIN_POST is issued, the sequence above
-    must complete, without further dependencies.  */
+int Win::start(MPI_Group group, int /*assert*/)
+{
+  /* From MPI forum advices
+  The call to MPI_WIN_COMPLETE does not return until the put call has completed at the origin; and the target window
+  will be accessed by the put operation only after the call to MPI_WIN_START has matched a call to MPI_WIN_POST by
+  the target process. This still leaves much choice to implementors. The call to MPI_WIN_START can block until the
+  matching call to MPI_WIN_POST occurs at all target processes. One can also have implementations where the call to
+  MPI_WIN_START is nonblocking, but the call to MPI_PUT blocks until the matching call to MPI_WIN_POST occurred; or
+  implementations where the first two calls are nonblocking, but the call to MPI_WIN_COMPLETE blocks until the call
+  to MPI_WIN_POST occurred; or even implementations where all three calls can complete before any target process
+  called MPI_WIN_POST --- the data put must be buffered, in this last case, so as to allow the put to complete at the
+  origin ahead of its completion at the target. However, once the call to MPI_WIN_POST is issued, the sequence above
+  must complete, without further dependencies.  */
 
   //naive, blocking implementation.
-    int i             = 0;
-    int j             = 0;
-    int size          = group->size();
-    MPI_Request* reqs = xbt_new0(MPI_Request, size);
-
-    while (j != size) {
-      int src = group->index(j);
-      if (src != smpi_process()->index() && src != MPI_UNDEFINED) {
-        reqs[i] = Request::irecv_init(nullptr, 0, MPI_CHAR, src, SMPI_RMA_TAG + 4, MPI_COMM_WORLD);
-        i++;
-      }
-      j++;
+  int i             = 0;
+  int j             = 0;
+  int size          = group->size();
+  MPI_Request* reqs = xbt_new0(MPI_Request, size);
+
+  XBT_DEBUG("Entering MPI_Win_Start");
+  while (j != size) {
+    int src = comm_->group()->rank(group->actor(j));
+    if (src != rank_ && src != MPI_UNDEFINED) { // TODO cheinrich: The check of MPI_UNDEFINED should be useless here
+      reqs[i] = Request::irecv_init(nullptr, 0, MPI_CHAR, src, SMPI_RMA_TAG + 4, comm_);
+      i++;
+    }
+    j++;
   }
-  size=i;
+  size = i;
   Request::startall(size, reqs);
   Request::waitall(size, reqs, MPI_STATUSES_IGNORE);
-  for(i=0;i<size;i++){
+  for (i = 0; i < size; i++) {
     Request::unref(&reqs[i]);
   }
   xbt_free(reqs);
   opened_++; //we're open for business !
   group_=group;
   group->ref();
+  XBT_DEBUG("Leaving MPI_Win_Start");
   return MPI_SUCCESS;
 }
 
-int Win::post(MPI_Group group, int assert){
+int Win::post(MPI_Group group, int /*assert*/)
+{
   //let's make a synchronous send here
   int i             = 0;
   int j             = 0;
   int size = group->size();
   MPI_Request* reqs = xbt_new0(MPI_Request, size);
 
+  XBT_DEBUG("Entering MPI_Win_Post");
   while(j!=size){
-    int dst=group->index(j);
-    if(dst!=smpi_process()->index() && dst!=MPI_UNDEFINED){
-      reqs[i]=Request::send_init(nullptr, 0, MPI_CHAR, dst, SMPI_RMA_TAG+4, MPI_COMM_WORLD);
+    int dst = comm_->group()->rank(group->actor(j));
+    if (dst != rank_ && dst != MPI_UNDEFINED) {
+      reqs[i] = Request::send_init(nullptr, 0, MPI_CHAR, dst, SMPI_RMA_TAG + 4, comm_);
       i++;
     }
     j++;
@@ -493,6 +506,7 @@ int Win::post(MPI_Group group, int assert){
   opened_++; //we're open for business !
   group_=group;
   group->ref();
+  XBT_DEBUG("Leaving MPI_Win_Post");
   return MPI_SUCCESS;
 }
 
@@ -503,13 +517,13 @@ int Win::complete(){
   XBT_DEBUG("Entering MPI_Win_Complete");
   int i             = 0;
   int j             = 0;
-  int size = group_->size();
+  int size          = group_->size();
   MPI_Request* reqs = xbt_new0(MPI_Request, size);
 
   while(j!=size){
-    int dst=group_->index(j);
-    if(dst!=smpi_process()->index() && dst!=MPI_UNDEFINED){
-      reqs[i]=Request::send_init(nullptr, 0, MPI_CHAR, dst, SMPI_RMA_TAG+5, MPI_COMM_WORLD);
+    int dst = comm_->group()->rank(group_->actor(j));
+    if (dst != rank_ && dst != MPI_UNDEFINED) {
+      reqs[i] = Request::send_init(nullptr, 0, MPI_CHAR, dst, SMPI_RMA_TAG + 5, comm_);
       i++;
     }
     j++;
@@ -541,9 +555,9 @@ int Win::wait(){
   MPI_Request* reqs = xbt_new0(MPI_Request, size);
 
   while(j!=size){
-    int src=group_->index(j);
-    if(src!=smpi_process()->index() && src!=MPI_UNDEFINED){
-      reqs[i]=Request::irecv_init(nullptr, 0, MPI_CHAR, src,SMPI_RMA_TAG+5, MPI_COMM_WORLD);
+    int src = comm_->group()->rank(group_->actor(j));
+    if (src != rank_ && src != MPI_UNDEFINED) {
+      reqs[i] = Request::irecv_init(nullptr, 0, MPI_CHAR, src, SMPI_RMA_TAG + 5, comm_);
       i++;
     }
     j++;
@@ -564,14 +578,15 @@ int Win::wait(){
   return MPI_SUCCESS;
 }
 
-int Win::lock(int lock_type, int rank, int assert){
+int Win::lock(int lock_type, int rank, int /*assert*/)
+{
   MPI_Win target_win = connected_wins_[rank];
 
   if ((lock_type == MPI_LOCK_EXCLUSIVE && target_win->mode_ != MPI_LOCK_SHARED)|| target_win->mode_ == MPI_LOCK_EXCLUSIVE){
-    xbt_mutex_acquire(target_win->lock_mut_);
+    target_win->lock_mut_->lock();
     target_win->mode_+= lock_type;//add the lock_type to differentiate case when we are switching from EXCLUSIVE to SHARED (no release needed in the unlock)
     if(lock_type == MPI_LOCK_SHARED){//the window used to be exclusive, it's now shared.
-      xbt_mutex_release(target_win->lock_mut_);
+      target_win->lock_mut_->unlock();
    }
   } else if (not(target_win->mode_ == MPI_LOCK_SHARED && lock_type == MPI_LOCK_EXCLUSIVE))
     target_win->mode_ += lock_type; // don't set to exclusive if it's already shared
@@ -602,7 +617,7 @@ int Win::unlock(int rank){
   target_win->mode_= 0;
   target_win->lockers_.remove(comm_->rank());
   if (target_mode==MPI_LOCK_EXCLUSIVE){
-    xbt_mutex_release(target_win->lock_mut_);
+    target_win->lock_mut_->unlock();
   }
 
   int finished = finish_comms(rank);
@@ -616,18 +631,18 @@ int Win::unlock_all(){
   int i=0;
   int retval = MPI_SUCCESS;
   for (i=0; i<comm_->size();i++){
-      int ret = this->unlock(i);
-      if(ret != MPI_SUCCESS)
-        retval = ret;
+    int ret = this->unlock(i);
+    if (ret != MPI_SUCCESS)
+      retval = ret;
   }
   return retval;
 }
 
 int Win::flush(int rank){
   MPI_Win target_win = connected_wins_[rank];
-  int finished = finish_comms(rank);
+  int finished       = finish_comms(rank_);
   XBT_DEBUG("Win_flush on local %d - Finished %d RMA calls", rank_, finished);
-  finished = target_win->finish_comms(rank_);
+  finished = target_win->finish_comms(rank);
   XBT_DEBUG("Win_flush on remote %d - Finished %d RMA calls", rank, finished);
   return MPI_SUCCESS;
 }
@@ -639,11 +654,9 @@ int Win::flush_local(int rank){
 }
 
 int Win::flush_all(){
-  int i=0;
-  int finished = 0;
-  finished = finish_comms();
+  int finished = finish_comms();
   XBT_DEBUG("Win_flush_all on local - Finished %d RMA calls", finished);
-  for (i=0; i<comm_->size();i++){
+  for (int i = 0; i < comm_->size(); i++) {
     finished = connected_wins_[i]->finish_comms(rank_);
     XBT_DEBUG("Win_flush_all on %d - Finished %d RMA calls", i, finished);
   }
@@ -660,9 +673,8 @@ Win* Win::f2c(int id){
   return static_cast<Win*>(F2C::f2c(id));
 }
 
-
 int Win::finish_comms(){
-  xbt_mutex_acquire(mut_);
+  mut_->lock();
   //Finish own requests
   std::vector<MPI_Request> *reqqs = requests_;
   int size = static_cast<int>(reqqs->size());
@@ -671,12 +683,12 @@ int Win::finish_comms(){
     Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
     reqqs->clear();
   }
-  xbt_mutex_release(mut_);
+  mut_->unlock();
   return size;
 }
 
 int Win::finish_comms(int rank){
-  xbt_mutex_acquire(mut_);
+  mut_->lock();
   //Finish own requests
   std::vector<MPI_Request> *reqqs = requests_;
   int size = static_cast<int>(reqqs->size());
@@ -684,8 +696,13 @@ int Win::finish_comms(int rank){
     size = 0;
     std::vector<MPI_Request> myreqqs;
     std::vector<MPI_Request>::iterator iter = reqqs->begin();
+    int proc_id                             = comm_->group()->actor(rank)->get_pid();
     while (iter != reqqs->end()){
-      if(((*iter)!=MPI_REQUEST_NULL) && (((*iter)->src() == rank) || ((*iter)->dst() == rank))){
+      // Let's see if we're either the destination or the sender of this request
+      // because we only wait for requests that we are responsible for.
+      // Also use the process id here since the request itself returns from src()
+      // and dst() the process id, NOT the rank (which only exists in the context of a communicator).
+      if (((*iter) != MPI_REQUEST_NULL) && (((*iter)->src() == proc_id) || ((*iter)->dst() == proc_id))) {
         myreqqs.push_back(*iter);
         iter = reqqs->erase(iter);
         size++;
@@ -699,10 +716,37 @@ int Win::finish_comms(int rank){
       myreqqs.clear();
     }
   }
-  xbt_mutex_release(mut_);
+  mut_->unlock();
   return size;
 }
 
+int Win::shared_query(int rank, MPI_Aint* size, int* disp_unit, void* baseptr)
+{
+  MPI_Win target_win = rank != MPI_PROC_NULL ? connected_wins_[rank] : nullptr;
+  for (int i = 0; not target_win && i < comm_->size(); i++) {
+    if (connected_wins_[i]->size_ > 0)
+      target_win = connected_wins_[i];
+  }
+  if (target_win) {
+    *size                         = target_win->size_;
+    *disp_unit                    = target_win->disp_unit_;
+    *static_cast<void**>(baseptr) = target_win->base_;
+  } else {
+    *size                         = 0;
+    *static_cast<void**>(baseptr) = xbt_malloc(0);
+  }
+  return MPI_SUCCESS;
+}
+
+MPI_Errhandler Win::errhandler(){
+  return errhandler_;
+}
+
+void Win::set_errhandler(MPI_Errhandler errhandler){
+  errhandler_=errhandler;
+  if(errhandler_!= MPI_ERRHANDLER_NULL)
+    errhandler->ref();
+}
 
 }
 }