Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
Update copyright lines for 2022.
[simgrid.git] / src / smpi / mpi / smpi_win.cpp
index f62ab4e..57026f0 100644 (file)
@@ -1,4 +1,4 @@
-/* Copyright (c) 2007-2020. The SimGrid Team. All rights reserved.          */
+/* Copyright (c) 2007-2022. 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 "smpi_request.hpp"
 #include "src/smpi/include/smpi_actor.hpp"
 
+#include <algorithm>
+
 XBT_LOG_NEW_DEFAULT_SUBCATEGORY(smpi_rma, smpi, "Logging specific to SMPI (RMA operations)");
 
+#define CHECK_RMA_REMOTE_WIN(fun, win)\
+  if(target_count*target_datatype->get_extent()>win->size_){\
+    XBT_WARN("%s: Trying to move %zd, which exceeds the window size on target process %d : %zd - Bailing out.",\
+    fun, target_count*target_datatype->get_extent(), target_rank, win->size_);\
+    simgrid::smpi::utils::set_current_buffer(1,"win_base",win->base_);\
+    return MPI_ERR_RMA_RANGE;\
+  }
+
+#define CHECK_WIN_LOCKED(win)                                                                                          \
+  if (opened_ == 0) { /*check that post/start has been done*/                                                          \
+    bool locked = std::any_of(begin(win->lockers_), end(win->lockers_), [this](int it) { return it == this->rank_; }); \
+    if (not locked)                                                                                                    \
+      return MPI_ERR_WIN;                                                                                              \
+  }
 
 namespace simgrid{
 namespace smpi{
 std::unordered_map<int, smpi_key_elem> Win::keyvals_;
 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)
+Win::Win(void* base, MPI_Aint size, int disp_unit, MPI_Info info, MPI_Comm comm, bool allocated, bool dynamic)
     : base_(base)
     , size_(size)
     , disp_unit_(disp_unit)
     , info_(info)
     , comm_(comm)
+    , connected_wins_(comm->size())
     , rank_(comm->rank())
     , allocated_(allocated)
     , dynamic_(dynamic)
@@ -35,46 +52,29 @@ Win::Win(void* base, MPI_Aint size, int disp_unit, MPI_Info info, MPI_Comm comm,
   XBT_DEBUG("Creating window");
   if(info!=MPI_INFO_NULL)
     info->ref();
-  int comm_size          = 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;
   if(rank_==0){
-    bar_ = new s4u::Barrier(comm_size);
+    bar_ = new s4u::Barrier(comm->size());
   }
-  mode_=0;
-  errhandler_=MPI_ERRORS_ARE_FATAL;
   errhandler_->ref();
   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::allgather(&connected_wins_[rank_], sizeof(MPI_Win), MPI_BYTE, connected_wins_.data(), sizeof(MPI_Win),
+                   MPI_BYTE, comm);
 
-  colls::bcast(&(bar_), sizeof(s4u::Barrier*), MPI_BYTE, 0, comm);
+  colls::bcast(&bar_, sizeof(s4u::Barrier*), MPI_BYTE, 0, comm);
 
   colls::barrier(comm);
+  this->add_f();
 }
 
 Win::~Win(){
   //As per the standard, perform a barrier to ensure every async comm is finished
   bar_->wait();
 
-  int finished = finish_comms();
-  XBT_DEBUG("Win destructor - Finished %d RMA calls", finished);
+  flush_local_all();
 
-  delete requests_;
-  delete[] connected_wins_;
-  if (name_ != nullptr){
-    xbt_free(name_);
-  }
   if (info_ != MPI_INFO_NULL)
     simgrid::smpi::Info::unref(info_);
   if (errhandler_ != MPI_ERRHANDLER_NULL)
@@ -84,21 +84,22 @@ Win::~Win(){
 
   colls::barrier(comm_);
   Comm::unref(comm_);
-  
+
   if (rank_ == 0)
     delete bar_;
 
-  if(allocated_ !=0)
+  if (allocated_)
     xbt_free(base_);
 
+  F2C::free_f(this->f2c_id());
   cleanup_attr<Win>();
 }
 
 int Win::attach(void* /*base*/, MPI_Aint size)
 {
-  if (not(base_ == MPI_BOTTOM || base_ == 0))
+  if (not(base_ == MPI_BOTTOM || base_ == nullptr))
     return MPI_ERR_ARG;
-  base_=0;//actually the address will be given in the RMA calls, as being the disp.
+  base_ = nullptr; // actually the address will be given in the RMA calls, as being the disp.
   size_+=size;
   return MPI_SUCCESS;
 }
@@ -112,13 +113,11 @@ int Win::detach(const void* /*base*/)
 
 void Win::get_name(char* name, int* length) const
 {
-  if(name_==nullptr){
-    *length=0;
-    name=nullptr;
-    return;
+  *length = static_cast<int>(name_.length());
+  if (not name_.empty()) {
+    name_.copy(name, *length);
+    name[*length] = '\0';
   }
-  *length = strlen(name_);
-  strncpy(name, name_, *length+1);
 }
 
 void Win::get_group(MPI_Group* group){
@@ -131,9 +130,6 @@ void Win::get_group(MPI_Group* group){
 
 MPI_Info Win::info()
 {
-  if (info_ == MPI_INFO_NULL)
-    info_ = new Info();
-  info_->ref();
   return info_;
 }
 
@@ -142,6 +138,11 @@ int Win::rank() const
   return rank_;
 }
 
+MPI_Comm Win::comm() const
+{
+  return comm_;
+}
+
 MPI_Aint Win::size() const
 {
   return size_;
@@ -157,7 +158,7 @@ int Win::disp_unit() const
   return disp_unit_;
 }
 
-int Win::dynamic() const
+bool Win::dynamic() const
 {
   return dynamic_;
 }
@@ -172,34 +173,21 @@ void Win::set_info(MPI_Info info)
 }
 
 void Win::set_name(const char* name){
-  name_ = xbt_strdup(name);
+  name_ = name;
 }
 
 int Win::fence(int assert)
 {
   XBT_DEBUG("Entering fence");
-  if (opened_ == 0)
-    opened_=1;
-  if (assert != MPI_MODE_NOPRECEDE) {
+  opened_++;
+  if (not (assert & MPI_MODE_NOPRECEDE)) {
     // This is not the first fence => finalize what came before
     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 redimensioned when another process pushes.
-    // This would result in the array used by Request::waitall() to be invalidated.
-    // Another solution would be to copy the data and cleanup the vector *before* Request::waitall
-    std::vector<MPI_Request> *reqs = requests_;
-    int size = static_cast<int>(reqs->size());
-    // start all requests that have been prepared by another process
-    if (size > 0) {
-      MPI_Request* treqs = &(*reqs)[0];
-      Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
-    }
+    flush_local_all();
     count_=0;
-    mut_->unlock();
   }
 
-  if(assert==MPI_MODE_NOSUCCEED)//there should be no ops after this one, tell we are closed.
+  if (assert & MPI_MODE_NOSUCCEED) // there should be no ops after this one, tell we are closed.
     opened_=0;
   assert_ = assert;
 
@@ -213,35 +201,23 @@ int Win::put(const void *origin_addr, int origin_count, MPI_Datatype origin_data
               MPI_Aint target_disp, int target_count, MPI_Datatype target_datatype, MPI_Request* request)
 {
   //get receiver pointer
-  const 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 const& it : recv_win->lockers_)
-      if (it == comm_->rank())
-        locked = 1;
-    if(locked != 1)
-      return MPI_ERR_WIN;
-  }
+  Win* recv_win = connected_wins_[target_rank];
 
-  if(target_count*target_datatype->get_extent()>recv_win->size_)
-    return MPI_ERR_ARG;
+  CHECK_WIN_LOCKED(recv_win)
+  CHECK_RMA_REMOTE_WIN("MPI_Put", recv_win)
 
-  void* recv_addr = static_cast<void*> ( static_cast<char*>(recv_win->base_) + target_disp * recv_win->disp_unit_);
+  void* recv_addr = static_cast<char*>(recv_win->base_) + target_disp * recv_win->disp_unit_;
 
-  if (target_rank != comm_->rank()) { // This is not for myself, so we need to send messages
+  if (target_rank != 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);
+        Request::rma_send_init(origin_addr, origin_count, origin_datatype, rank_, target_rank, SMPI_RMA_TAG + 1, comm_,
+                               MPI_OP_NULL);
 
     //prepare receiver request
-    // 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);
+    MPI_Request rreq = Request::rma_recv_init(recv_addr, target_count, target_datatype, rank_, target_rank,
+                                              SMPI_RMA_TAG + 1, recv_win->comm_, MPI_OP_NULL);
 
     //start send
     sreq->start();
@@ -250,13 +226,13 @@ int Win::put(const void *origin_addr, int origin_count, MPI_Datatype origin_data
       *request=sreq;
     }else{
       mut_->lock();
-      requests_->push_back(sreq);
+      requests_.push_back(sreq);
       mut_->unlock();
     }
 
     //push request to receiver's win
     recv_win->mut_->lock();
-    recv_win->requests_->push_back(rreq);
+    recv_win->requests_.push_back(rreq);
     rreq->start();
     recv_win->mut_->unlock();
   } else {
@@ -273,40 +249,28 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
               MPI_Aint target_disp, int target_count, MPI_Datatype target_datatype, MPI_Request* request)
 {
   //get sender pointer
-  const Win* send_win = connected_wins_[target_rank];
+  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 const& it : send_win->lockers_)
-      if (it == comm_->rank())
-        locked = 1;
-    if(locked != 1)
-      return MPI_ERR_WIN;
-  }
-
-  if(target_count*target_datatype->get_extent()>send_win->size_)
-    return MPI_ERR_ARG;
+  CHECK_WIN_LOCKED(send_win)
+  CHECK_RMA_REMOTE_WIN("MPI_Get", send_win)
 
   const 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 != comm_->rank()){
+  if (target_rank != rank_) {
     //prepare send_request
-    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);
+    MPI_Request sreq = Request::rma_send_init(send_addr, target_count, target_datatype, target_rank, 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, 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);
+    MPI_Request rreq = Request::rma_recv_init(origin_addr, origin_count, origin_datatype, target_rank, rank_,
+                                              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
+    // push request to sender's win
     send_win->mut_->lock();
-    send_win->requests_->push_back(sreq);
+    send_win->requests_.push_back(sreq);
     send_win->mut_->unlock();
 
     //start recv
@@ -316,7 +280,7 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
       *request=rreq;
     }else{
       mut_->lock();
-      requests_->push_back(rreq);
+      requests_.push_back(rreq);
       mut_->unlock();
     }
   } else {
@@ -332,34 +296,24 @@ int Win::accumulate(const void *origin_addr, int origin_count, MPI_Datatype orig
 {
   XBT_DEBUG("Entering MPI_Win_Accumulate");
   //get receiver pointer
-  const 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 const& it : recv_win->lockers_)
-      if (it == comm_->rank())
-        locked = 1;
-    if(locked != 1)
-      return MPI_ERR_WIN;
-  }
-  //FIXME: local version
+  Win* recv_win = connected_wins_[target_rank];
 
-  if(target_count*target_datatype->get_extent()>recv_win->size_)
-    return MPI_ERR_ARG;
+  //FIXME: local version
+  CHECK_WIN_LOCKED(recv_win)
+  CHECK_RMA_REMOTE_WIN("MPI_Accumulate", recv_win)
 
-  void* recv_addr = static_cast<void*>(static_cast<char*>(recv_win->base_) + target_disp * recv_win->disp_unit_);
+  void* recv_addr = static_cast<char*>(recv_win->base_) + target_disp * recv_win->disp_unit_;
   XBT_DEBUG("Entering MPI_Accumulate to %d", target_rank);
   // As the tag will be used for ordering of the operations, subtract 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, comm_->rank(), target_rank,
+  MPI_Request sreq = Request::rma_send_init(origin_addr, origin_count, origin_datatype, 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, recv_win->comm_->rank(),
-                                            recv_win->comm_->group()->rank(comm_->group()->actor(target_rank)), SMPI_RMA_TAG - 3 - count_, recv_win->comm_, op);
+  MPI_Request rreq = Request::rma_recv_init(recv_addr, target_count, target_datatype, rank_, target_rank,
+                                            SMPI_RMA_TAG - 3 - count_, recv_win->comm_, op);
 
   count_++;
 
@@ -367,7 +321,7 @@ int Win::accumulate(const void *origin_addr, int origin_count, MPI_Datatype orig
   sreq->start();
   // push request to receiver's win
   recv_win->mut_->lock();
-  recv_win->requests_->push_back(rreq);
+  recv_win->requests_.push_back(rreq);
   rreq->start();
   recv_win->mut_->unlock();
 
@@ -375,10 +329,13 @@ int Win::accumulate(const void *origin_addr, int origin_count, MPI_Datatype orig
     *request = sreq;
   } else {
     mut_->lock();
-    requests_->push_back(sreq);
+    requests_.push_back(sreq);
     mut_->unlock();
   }
 
+  // FIXME: The current implementation fails to ensure the correct ordering of the accumulate requests.  The following
+  // 'flush' is a workaround to fix that.
+  flush(target_rank);
   XBT_DEBUG("Leaving MPI_Win_Accumulate");
   return MPI_SUCCESS;
 }
@@ -390,22 +347,12 @@ int Win::get_accumulate(const void* origin_addr, int origin_count, MPI_Datatype
   //get sender pointer
   const 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 const& it : send_win->lockers_)
-      if (it == comm_->rank())
-        locked = 1;
-    if(locked != 1)
-      return MPI_ERR_WIN;
-  }
-
-  if(target_count*target_datatype->get_extent()>send_win->size_)
-    return MPI_ERR_ARG;
+  CHECK_WIN_LOCKED(send_win)
+  CHECK_RMA_REMOTE_WIN("MPI_Get_Accumulate", send_win)
 
   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;
+  MPI_Request req = MPI_REQUEST_NULL;
   send_win->atomic_mut_->lock();
   get(result_addr, result_count, result_datatype, target_rank,
               target_disp, target_count, target_datatype, &req);
@@ -426,15 +373,7 @@ int Win::compare_and_swap(const void* origin_addr, const void* compare_addr, voi
   //get sender pointer
   const 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 const& it : send_win->lockers_)
-      if (it == comm_->rank())
-        locked = 1;
-    if(locked != 1)
-      return MPI_ERR_WIN;
-  }
+  CHECK_WIN_LOCKED(send_win)
 
   XBT_DEBUG("Entering MPI_Compare_and_swap with %d", target_rank);
   MPI_Request req = MPI_REQUEST_NULL;
@@ -466,30 +405,24 @@ int Win::start(MPI_Group group, int /*assert*/)
   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);
-
   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;
-  Request::startall(size, reqs);
-  Request::waitall(size, reqs, MPI_STATUSES_IGNORE);
-  for (i = 0; i < size; i++) {
-    Request::unref(&reqs[i]);
+  std::vector<MPI_Request> reqs;
+  for (int i = 0; i < group->size(); i++) {
+    int src = comm_->group()->rank(group->actor(i));
+    xbt_assert(src != MPI_UNDEFINED);
+    if (src != rank_)
+      reqs.emplace_back(Request::irecv_init(nullptr, 0, MPI_CHAR, src, SMPI_RMA_TAG + 4, comm_));
   }
-  xbt_free(reqs);
-  opened_++; //we're open for business !
-  group_=group;
+  int size = static_cast<int>(reqs.size());
+
+  Request::startall(size, reqs.data());
+  Request::waitall(size, reqs.data(), MPI_STATUSES_IGNORE);
+  for (auto& req : reqs)
+    Request::unref(&req);
+
   group->ref();
+  dst_group_ = group;
+  opened_++; // we're open for business !
   XBT_DEBUG("Leaving MPI_Win_Start");
   return MPI_SUCCESS;
 }
@@ -497,100 +430,78 @@ int Win::start(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 = 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++;
+  std::vector<MPI_Request> reqs;
+  for (int i = 0; i < group->size(); i++) {
+    int dst = comm_->group()->rank(group->actor(i));
+    xbt_assert(dst != MPI_UNDEFINED);
+    if (dst != rank_)
+      reqs.emplace_back(Request::send_init(nullptr, 0, MPI_CHAR, dst, SMPI_RMA_TAG + 4, comm_));
   }
-  size=i;
+  int size = static_cast<int>(reqs.size());
+
+  Request::startall(size, reqs.data());
+  Request::waitall(size, reqs.data(), MPI_STATUSES_IGNORE);
+  for (auto& req : reqs)
+    Request::unref(&req);
 
-  Request::startall(size, reqs);
-  Request::waitall(size, reqs, MPI_STATUSES_IGNORE);
-  for(i=0;i<size;i++){
-    Request::unref(&reqs[i]);
-  }
-  xbt_free(reqs);
-  opened_++; //we're open for business !
-  group_=group;
   group->ref();
+  src_group_ = group;
+  opened_++; // we're open for business !
   XBT_DEBUG("Leaving MPI_Win_Post");
   return MPI_SUCCESS;
 }
 
 int Win::complete(){
-  if(opened_==0)
-    xbt_die("Complete called on already opened MPI_Win");
+  xbt_assert(opened_ != 0, "Complete called on already opened MPI_Win");
 
   XBT_DEBUG("Entering MPI_Win_Complete");
-  int i             = 0;
-  int j             = 0;
-  int size          = group_->size();
-  MPI_Request* reqs = xbt_new0(MPI_Request, size);
-
-  while(j!=size){
-    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++;
+  std::vector<MPI_Request> reqs;
+  for (int i = 0; i < dst_group_->size(); i++) {
+    int dst = comm_->group()->rank(dst_group_->actor(i));
+    xbt_assert(dst != MPI_UNDEFINED);
+    if (dst != rank_)
+      reqs.emplace_back(Request::send_init(nullptr, 0, MPI_CHAR, dst, SMPI_RMA_TAG + 5, comm_));
   }
-  size=i;
-  XBT_DEBUG("Win_complete - Sending sync messages to %d processes", size);
-  Request::startall(size, reqs);
-  Request::waitall(size, reqs, MPI_STATUSES_IGNORE);
+  int size = static_cast<int>(reqs.size());
 
-  for(i=0;i<size;i++){
-    Request::unref(&reqs[i]);
-  }
-  xbt_free(reqs);
+  XBT_DEBUG("Win_complete - Sending sync messages to %d processes", size);
+  Request::startall(size, reqs.data());
+  Request::waitall(size, reqs.data(), MPI_STATUSES_IGNORE);
+  for (auto& req : reqs)
+    Request::unref(&req);
 
-  int finished = finish_comms();
-  XBT_DEBUG("Win_complete - Finished %d RMA calls", finished);
+  flush_local_all();
 
-  Group::unref(group_);
   opened_--; //we're closed for business !
+  Group::unref(dst_group_);
+  dst_group_ = MPI_GROUP_NULL;
   return MPI_SUCCESS;
 }
 
 int Win::wait(){
   //naive, blocking implementation.
   XBT_DEBUG("Entering MPI_Win_Wait");
-  int i             = 0;
-  int j             = 0;
-  int size          = group_->size();
-  MPI_Request* reqs = xbt_new0(MPI_Request, size);
-
-  while(j!=size){
-    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++;
+  std::vector<MPI_Request> reqs;
+  for (int i = 0; i < src_group_->size(); i++) {
+    int src = comm_->group()->rank(src_group_->actor(i));
+    xbt_assert(src != MPI_UNDEFINED);
+    if (src != rank_)
+      reqs.emplace_back(Request::irecv_init(nullptr, 0, MPI_CHAR, src, SMPI_RMA_TAG + 5, comm_));
   }
-  size=i;
+  int size = static_cast<int>(reqs.size());
+
   XBT_DEBUG("Win_wait - Receiving sync messages from %d processes", size);
-  Request::startall(size, reqs);
-  Request::waitall(size, reqs, MPI_STATUSES_IGNORE);
-  for(i=0;i<size;i++){
-    Request::unref(&reqs[i]);
-  }
-  xbt_free(reqs);
-  int finished = finish_comms();
-  XBT_DEBUG("Win_wait - Finished %d RMA calls", finished);
+  Request::startall(size, reqs.data());
+  Request::waitall(size, reqs.data(), MPI_STATUSES_IGNORE);
+  for (auto& req : reqs)
+    Request::unref(&req);
+
+  flush_local_all();
 
-  Group::unref(group_);
-  opened_--; //we're opened for business !
+  opened_--; //we're closed for business !
+  Group::unref(src_group_);
+  src_group_ = MPI_GROUP_NULL;
   return MPI_SUCCESS;
 }
 
@@ -607,22 +518,18 @@ int Win::lock(int lock_type, int rank, int /*assert*/)
   } 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
 
-  target_win->lockers_.push_back(comm_->rank());
+  target_win->lockers_.push_back(rank_);
 
-  int finished = finish_comms(rank);
-  XBT_DEBUG("Win_lock %d - Finished %d RMA calls", rank, finished);
-  finished = target_win->finish_comms(rank_);
-  XBT_DEBUG("Win_lock target %d - Finished %d RMA calls", rank, finished);
+  flush(rank);
   return MPI_SUCCESS;
 }
 
 int Win::lock_all(int assert){
-  int i=0;
   int retval = MPI_SUCCESS;
-  for (i=0; i<comm_->size();i++){
-      int ret = this->lock(MPI_LOCK_SHARED, i, assert);
-      if(ret != MPI_SUCCESS)
-        retval = ret;
+  for (int i = 0; i < comm_->size(); i++) {
+    int ret = this->lock(MPI_LOCK_SHARED, i, assert);
+    if (ret != MPI_SUCCESS)
+      retval = ret;
   }
   return retval;
 }
@@ -631,22 +538,18 @@ int Win::unlock(int rank){
   MPI_Win target_win = connected_wins_[rank];
   int target_mode = target_win->mode_;
   target_win->mode_= 0;
-  target_win->lockers_.remove(comm_->rank());
+  target_win->lockers_.remove(rank_);
   if (target_mode==MPI_LOCK_EXCLUSIVE){
     target_win->lock_mut_->unlock();
   }
 
-  int finished = finish_comms(rank);
-  XBT_DEBUG("Win_unlock %d - Finished %d RMA calls", rank, finished);
-  finished = target_win->finish_comms(rank_);
-  XBT_DEBUG("Win_unlock target %d - Finished %d RMA calls", rank, finished);
+  flush(rank);
   return MPI_SUCCESS;
 }
 
 int Win::unlock_all(){
-  int i=0;
   int retval = MPI_SUCCESS;
-  for (i=0; i<comm_->size();i++){
+  for (int i = 0; i < comm_->size(); i++) {
     int ret = this->unlock(i);
     if (ret != MPI_SUCCESS)
       retval = ret;
@@ -655,33 +558,36 @@ int Win::unlock_all(){
 }
 
 int Win::flush(int rank){
-  MPI_Win target_win = connected_wins_[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);
-  XBT_DEBUG("Win_flush on remote %d - Finished %d RMA calls", rank, finished);
+  int finished = finish_comms(rank);
+  XBT_DEBUG("Win_flush on local %d for remote %d - Finished %d RMA calls", rank_, rank, finished);
+  if (rank != rank_) {
+    finished = connected_wins_[rank]->finish_comms(rank_);
+    XBT_DEBUG("Win_flush on remote %d for local %d - Finished %d RMA calls", rank, rank_, finished);
+  }
   return MPI_SUCCESS;
 }
 
 int Win::flush_local(int rank){
   int finished = finish_comms(rank);
-  XBT_DEBUG("Win_flush_local for rank %d - Finished %d RMA calls", rank, finished);
+  XBT_DEBUG("Win_flush_local on local %d for remote %d - Finished %d RMA calls", rank_, rank, finished);
   return MPI_SUCCESS;
 }
 
 int Win::flush_all(){
   int finished = finish_comms();
-  XBT_DEBUG("Win_flush_all on local - Finished %d RMA calls", finished);
+  XBT_DEBUG("Win_flush_all on local %d - Finished %d RMA calls", rank_, finished);
   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);
+    if (i != rank_) {
+      finished = connected_wins_[i]->finish_comms(rank_);
+      XBT_DEBUG("Win_flush_all on remote %d for local %d - Finished %d RMA calls", i, rank_, finished);
+    }
   }
   return MPI_SUCCESS;
 }
 
 int Win::flush_local_all(){
   int finished = finish_comms();
-  XBT_DEBUG("Win_flush_local_all - Finished %d RMA calls", finished);
+  XBT_DEBUG("Win_flush_local_all on local %d - Finished %d RMA calls", rank_, finished);
   return MPI_SUCCESS;
 }
 
@@ -690,47 +596,41 @@ Win* Win::f2c(int id){
 }
 
 int Win::finish_comms(){
+  // This (simulated) mutex ensures that no process pushes to the vector of requests during the waitall.
+  // Without this, the vector could get redimensioned when another process pushes.
+  // This would result in the array used by Request::waitall() to be invalidated.
+  // Another solution would be to copy the data and cleanup the vector *before* Request::waitall
   mut_->lock();
   //Finish own requests
-  std::vector<MPI_Request> *reqqs = requests_;
-  int size = static_cast<int>(reqqs->size());
+  int size = static_cast<int>(requests_.size());
   if (size > 0) {
-    MPI_Request* treqs = &(*reqqs)[0];
+    MPI_Request* treqs = requests_.data();
     Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
-    reqqs->clear();
+    requests_.clear();
   }
   mut_->unlock();
   return size;
 }
 
 int Win::finish_comms(int rank){
+  // See comment about the mutex in finish_comms() above
   mut_->lock();
-  //Finish own requests
-  std::vector<MPI_Request> *reqqs = requests_;
-  int size = static_cast<int>(reqqs->size());
+  // Finish own requests
+  // 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).
+  aid_t proc_id = comm_->group()->actor(rank);
+  auto it     = std::stable_partition(begin(requests_), end(requests_), [proc_id](const MPI_Request& req) {
+    return (req == MPI_REQUEST_NULL || (req->src() != proc_id && req->dst() != proc_id));
+  });
+  std::vector<MPI_Request> myreqqs(it, end(requests_));
+  requests_.erase(it, end(requests_));
+  int size = static_cast<int>(myreqqs.size());
   if (size > 0) {
-    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()){
-      // 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++;
-      } else {
-        ++iter;
-      }
-    }
-    if(size >0){
-      MPI_Request* treqs = &myreqqs[0];
-      Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
-      myreqqs.clear();
-    }
+    MPI_Request* treqs = myreqqs.data();
+    Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
+    myreqqs.clear();
   }
   mut_->unlock();
   return size;