Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
[SMPI] Replay: C++-ify (all)gather action
[simgrid.git] / src / smpi / internals / smpi_replay.cpp
index d89a3fa..8f0caad 100644 (file)
@@ -100,6 +100,97 @@ public:
   }
 };
 
+class CollCommParser : public ActionArgParser {
+public:
+  double size;
+  double comm_size;
+  double comp_size;
+  int send_size;
+  int recv_size;
+  int root = 0;
+  MPI_Datatype datatype1 = MPI_DEFAULT_TYPE;
+  MPI_Datatype datatype2 = MPI_DEFAULT_TYPE;
+};
+
+class BcastArgParser : public CollCommParser {
+public:
+  void parse(simgrid::xbt::ReplayAction& action) override
+  {
+    CHECK_ACTION_PARAMS(action, 1, 2)
+    size = parse_double(action[2]);
+    root = (action.size() > 3) ? std::stoi(action[3]) : 0;
+    if (action.size() > 4)
+      datatype1 = simgrid::smpi::Datatype::decode(action[4]);
+  }
+};
+
+class ReduceArgParser : public CollCommParser {
+public:
+  void parse(simgrid::xbt::ReplayAction& action) override
+  {
+    CHECK_ACTION_PARAMS(action, 2, 2)
+    comm_size = parse_double(action[2]);
+    comp_size = parse_double(action[3]);
+    root      = (action.size() > 4) ? std::stoi(action[4]) : 0;
+    if (action.size() > 5)
+      datatype1 = simgrid::smpi::Datatype::decode(action[5]);
+  }
+};
+
+class AllReduceArgParser : public CollCommParser {
+public:
+  void parse(simgrid::xbt::ReplayAction& action) override
+  {
+    CHECK_ACTION_PARAMS(action, 2, 1)
+    comm_size = parse_double(action[2]);
+    comp_size = parse_double(action[3]);
+    if (action.size() > 4)
+      datatype1 = simgrid::smpi::Datatype::decode(action[4]);
+  }
+};
+
+class AllToAllArgParser : public CollCommParser {
+public:
+  void parse(simgrid::xbt::ReplayAction& action) override
+  {
+    CHECK_ACTION_PARAMS(action, 2, 1)
+    comm_size = MPI_COMM_WORLD->size();
+    send_size = parse_double(action[2]);
+    recv_size = parse_double(action[3]);
+
+    if (action.size() > 4)
+      datatype1 = simgrid::smpi::Datatype::decode(action[4]);
+    if (action.size() > 5)
+      datatype2 = simgrid::smpi::Datatype::decode(action[5]);
+  }
+};
+
+class GatherParser : public CollCommParser {
+public:
+  void parse(simgrid::xbt::ReplayAction& action) override
+  {
+    /* The structure of the gather action for the rank 0 (total 4 processes) is the following:
+          0 gather 68 68 0 0 0
+        where:
+          1) 68 is the sendcounts
+          2) 68 is the recvcounts
+          3) 0 is the root node
+          4) 0 is the send datatype id, see simgrid::smpi::Datatype::decode()
+          5) 0 is the recv datatype id, see simgrid::smpi::Datatype::decode()
+    */
+    CHECK_ACTION_PARAMS(action, 2, 3)
+    comm_size = MPI_COMM_WORLD->size();
+    send_size = parse_double(action[2]);
+    recv_size = parse_double(action[3]);
+    root      = (action.size() > 4) ? std::stoi(action[4]) : 0;
+
+    if (action.size() > 5)
+      datatype1 = simgrid::smpi::Datatype::decode(action[5]);
+    if (action.size() > 6)
+      datatype2 = simgrid::smpi::Datatype::decode(action[6]);
+  }
+};
+
 template <class T> class ReplayAction {
 protected:
   const std::string name;
@@ -121,6 +212,16 @@ public:
   }
 
   virtual void kernel(simgrid::xbt::ReplayAction& action) = 0;
+
+  void* send_buffer(int size)
+  {
+    return smpi_get_tmp_sendbuffer(size);
+  }
+
+  void* recv_buffer(int size)
+  {
+    return smpi_get_tmp_recvbuffer(size);
+  }
 };
 
 class WaitAction : public ReplayAction<ActionArgParser> {
@@ -128,9 +229,6 @@ public:
   WaitAction() : ReplayAction("Wait") {}
   void kernel(simgrid::xbt::ReplayAction& action) override
   {
-    CHECK_ACTION_PARAMS(action, 0, 0)
-    MPI_Status status;
-
     std::string s = boost::algorithm::join(action, " ");
     xbt_assert(get_reqq_self()->size(), "action wait not preceded by any irecv or isend: %s", s.c_str());
     MPI_Request request = get_reqq_self()->back();
@@ -152,6 +250,7 @@ public:
     // TODO: Here we take the rank while we normally take the process id (look for my_proc_id)
     TRACE_smpi_comm_in(rank, __FUNCTION__, new simgrid::instr::NoOpTIData("wait"));
 
+    MPI_Status status;
     Request::wait(&request, &status);
 
     TRACE_smpi_comm_out(rank);
@@ -318,137 +417,91 @@ public:
   }
 };
 
-} // Replay Namespace
-
-static void action_bcast(simgrid::xbt::ReplayAction& action)
-{
-  CHECK_ACTION_PARAMS(action, 1, 2)
-  double size = parse_double(action[2]);
-  double clock = smpi_process()->simulated_elapsed();
-  int root     = (action.size() > 3) ? std::stoi(action[3]) : 0;
-  /* Initialize MPI_CURRENT_TYPE in order to decrease the number of the checks */
-  MPI_Datatype MPI_CURRENT_TYPE = (action.size() > 4) ? simgrid::smpi::Datatype::decode(action[4]) : MPI_DEFAULT_TYPE;
-
-  int my_proc_id = Actor::self()->getPid();
-  TRACE_smpi_comm_in(my_proc_id, __FUNCTION__,
-                     new simgrid::instr::CollTIData("bcast", MPI_COMM_WORLD->group()->actor(root)->getPid(), -1.0, size,
-                                                    -1, Datatype::encode(MPI_CURRENT_TYPE), ""));
-
-  void *sendbuf = smpi_get_tmp_sendbuffer(size* MPI_CURRENT_TYPE->size());
-
-  Colls::bcast(sendbuf, size, MPI_CURRENT_TYPE, root, MPI_COMM_WORLD);
-
-  TRACE_smpi_comm_out(my_proc_id);
-  log_timed_action (action, clock);
-}
-
-static void action_reduce(simgrid::xbt::ReplayAction& action)
-{
-  CHECK_ACTION_PARAMS(action, 2, 2)
-  double comm_size = parse_double(action[2]);
-  double comp_size = parse_double(action[3]);
-  double clock = smpi_process()->simulated_elapsed();
-  int root         = (action.size() > 4) ? std::stoi(action[4]) : 0;
-
-  MPI_Datatype MPI_CURRENT_TYPE = (action.size() > 5) ? simgrid::smpi::Datatype::decode(action[5]) : MPI_DEFAULT_TYPE;
-
-  int my_proc_id = Actor::self()->getPid();
-  TRACE_smpi_comm_in(my_proc_id, __FUNCTION__,
-                     new simgrid::instr::CollTIData("reduce", MPI_COMM_WORLD->group()->actor(root)->getPid(), comp_size,
-                                                    comm_size, -1, Datatype::encode(MPI_CURRENT_TYPE), ""));
-
-  void *recvbuf = smpi_get_tmp_sendbuffer(comm_size* MPI_CURRENT_TYPE->size());
-  void *sendbuf = smpi_get_tmp_sendbuffer(comm_size* MPI_CURRENT_TYPE->size());
-  Colls::reduce(sendbuf, recvbuf, comm_size, MPI_CURRENT_TYPE, MPI_OP_NULL, root, MPI_COMM_WORLD);
-  smpi_execute_flops(comp_size);
-
-  TRACE_smpi_comm_out(my_proc_id);
-  log_timed_action (action, clock);
-}
-
-static void action_allReduce(simgrid::xbt::ReplayAction& action)
-{
-  CHECK_ACTION_PARAMS(action, 2, 1)
-  double comm_size = parse_double(action[2]);
-  double comp_size = parse_double(action[3]);
-
-  MPI_Datatype MPI_CURRENT_TYPE = (action.size() > 4) ? simgrid::smpi::Datatype::decode(action[4]) : MPI_DEFAULT_TYPE;
+class BcastAction : public ReplayAction<BcastArgParser> {
+public:
+  BcastAction() : ReplayAction("bcast") {}
+  void kernel(simgrid::xbt::ReplayAction& action) override
+  {
+    TRACE_smpi_comm_in(my_proc_id, "action_bcast",
+                       new simgrid::instr::CollTIData("bcast", MPI_COMM_WORLD->group()->actor(args.root)->getPid(),
+                                                      -1.0, args.size, -1, Datatype::encode(args.datatype1), ""));
 
-  double clock = smpi_process()->simulated_elapsed();
-  int my_proc_id = Actor::self()->getPid();
-  TRACE_smpi_comm_in(my_proc_id, __FUNCTION__, new simgrid::instr::CollTIData("allReduce", -1, comp_size, comm_size, -1,
-                                                                              Datatype::encode(MPI_CURRENT_TYPE), ""));
+    Colls::bcast(send_buffer(args.size * args.datatype1->size()), args.size, args.datatype1, args.root, MPI_COMM_WORLD);
 
-  void *recvbuf = smpi_get_tmp_recvbuffer(comm_size* MPI_CURRENT_TYPE->size());
-  void *sendbuf = smpi_get_tmp_sendbuffer(comm_size* MPI_CURRENT_TYPE->size());
-  Colls::allreduce(sendbuf, recvbuf, comm_size, MPI_CURRENT_TYPE, MPI_OP_NULL, MPI_COMM_WORLD);
-  smpi_execute_flops(comp_size);
+    TRACE_smpi_comm_out(my_proc_id);
+  }
+};
 
-  TRACE_smpi_comm_out(my_proc_id);
-  log_timed_action (action, clock);
-}
+class ReduceAction : public ReplayAction<ReduceArgParser> {
+public:
+  ReduceAction() : ReplayAction("reduce") {}
+  void kernel(simgrid::xbt::ReplayAction& action) override
+  {
+    TRACE_smpi_comm_in(my_proc_id, "action_reduce",
+                       new simgrid::instr::CollTIData("reduce", MPI_COMM_WORLD->group()->actor(args.root)->getPid(), args.comp_size,
+                                                      args.comm_size, -1, Datatype::encode(args.datatype1), ""));
 
-static void action_allToAll(simgrid::xbt::ReplayAction& action)
-{
-  CHECK_ACTION_PARAMS(action, 2, 2) //two mandatory (send and recv volumes) and two optional (corresponding datatypes)
-  double clock = smpi_process()->simulated_elapsed();
-  unsigned long comm_size = MPI_COMM_WORLD->size();
-  int send_size = parse_double(action[2]);
-  int recv_size = parse_double(action[3]);
-  MPI_Datatype MPI_CURRENT_TYPE{(action.size() > 5) ? simgrid::smpi::Datatype::decode(action[4]) : MPI_DEFAULT_TYPE};
-  MPI_Datatype MPI_CURRENT_TYPE2{(action.size() > 5) ? simgrid::smpi::Datatype::decode(action[5]) : MPI_DEFAULT_TYPE};
+    Colls::reduce(send_buffer(args.comm_size * args.datatype1->size()),
+        recv_buffer(args.comm_size * args.datatype1->size()), args.comm_size, args.datatype1, MPI_OP_NULL, args.root, MPI_COMM_WORLD);
+    smpi_execute_flops(args.comp_size);
 
-  void *send = smpi_get_tmp_sendbuffer(send_size*comm_size* MPI_CURRENT_TYPE->size());
-  void *recv = smpi_get_tmp_recvbuffer(recv_size*comm_size* MPI_CURRENT_TYPE2->size());
+    TRACE_smpi_comm_out(my_proc_id);
+  }
+};
 
-  int my_proc_id = Actor::self()->getPid();
-  TRACE_smpi_comm_in(my_proc_id, __FUNCTION__,
-                     new simgrid::instr::CollTIData("allToAll", -1, -1.0, send_size, recv_size,
-                                                    Datatype::encode(MPI_CURRENT_TYPE),
-                                                    Datatype::encode(MPI_CURRENT_TYPE2)));
+class AllReduceAction : public ReplayAction<AllReduceArgParser> {
+public:
+  AllReduceAction() : ReplayAction("allReduce") {}
+  void kernel(simgrid::xbt::ReplayAction& action) override
+  {
+    TRACE_smpi_comm_in(my_proc_id, "action_allReduce", new simgrid::instr::CollTIData("allReduce", -1, args.comp_size, args.comm_size, -1,
+                                                                                Datatype::encode(args.datatype1), ""));
 
-  Colls::alltoall(send, send_size, MPI_CURRENT_TYPE, recv, recv_size, MPI_CURRENT_TYPE2, MPI_COMM_WORLD);
+    Colls::allreduce(send_buffer(args.comm_size * args.datatype1->size()),
+        recv_buffer(args.comm_size * args.datatype1->size()), args.comm_size, args.datatype1, MPI_OP_NULL, MPI_COMM_WORLD);
+    smpi_execute_flops(args.comp_size);
 
-  TRACE_smpi_comm_out(my_proc_id);
-  log_timed_action (action, clock);
-}
+    TRACE_smpi_comm_out(my_proc_id);
+  }
+};
 
-static void action_gather(simgrid::xbt::ReplayAction& action)
-{
-  /* The structure of the gather action for the rank 0 (total 4 processes) is the following:
-        0 gather 68 68 0 0 0
-      where:
-        1) 68 is the sendcounts
-        2) 68 is the recvcounts
-        3) 0 is the root node
-        4) 0 is the send datatype id, see simgrid::smpi::Datatype::decode()
-        5) 0 is the recv datatype id, see simgrid::smpi::Datatype::decode()
-  */
-  CHECK_ACTION_PARAMS(action, 2, 3)
-  double clock = smpi_process()->simulated_elapsed();
-  unsigned long comm_size = MPI_COMM_WORLD->size();
-  int send_size = parse_double(action[2]);
-  int recv_size = parse_double(action[3]);
-  MPI_Datatype MPI_CURRENT_TYPE{(action.size() > 6) ? simgrid::smpi::Datatype::decode(action[5]) : MPI_DEFAULT_TYPE};
-  MPI_Datatype MPI_CURRENT_TYPE2{(action.size() > 6) ? simgrid::smpi::Datatype::decode(action[6]) : MPI_DEFAULT_TYPE};
+class AllToAllAction : public ReplayAction<AllToAllArgParser> {
+public:
+  AllToAllAction() : ReplayAction("allToAll") {}
+  void kernel(simgrid::xbt::ReplayAction& action) override
+  {
+    TRACE_smpi_comm_in(my_proc_id, "action_allToAll",
+                     new simgrid::instr::CollTIData("allToAll", -1, -1.0, args.send_size, args.recv_size,
+                                                    Datatype::encode(args.datatype1),
+                                                    Datatype::encode(args.datatype2)));
 
-  void *send = smpi_get_tmp_sendbuffer(send_size* MPI_CURRENT_TYPE->size());
-  void *recv = nullptr;
-  int root   = (action.size() > 4) ? std::stoi(action[4]) : 0;
-  int rank = MPI_COMM_WORLD->rank();
+    Colls::alltoall(send_buffer(args.send_size*args.comm_size* args.datatype1->size()), 
+      args.send_size, args.datatype1, recv_buffer(args.recv_size * args.comm_size * args.datatype2->size()),
+      args.recv_size, args.datatype2, MPI_COMM_WORLD);
 
-  if(rank==root)
-    recv = smpi_get_tmp_recvbuffer(recv_size*comm_size* MPI_CURRENT_TYPE2->size());
+    TRACE_smpi_comm_out(my_proc_id);
+  }
+};
 
-  TRACE_smpi_comm_in(rank, __FUNCTION__, new simgrid::instr::CollTIData("gather", root, -1.0, send_size, recv_size,
-                                                                        Datatype::encode(MPI_CURRENT_TYPE),
-                                                                        Datatype::encode(MPI_CURRENT_TYPE2)));
+class GatherAction : public ReplayAction<GatherArgParser> {
+public:
+  GatherAction(std::string name) : ReplayAction(name) {}
+  void kernel(simgrid::xbt::ReplayAction& action) override
+  {
+    TRACE_smpi_comm_in(my_proc_id, name, new simgrid::instr::CollTIData(name, (name == "gather") ? args.root : -1, -1.0, args.send_size, args.recv_size,
+                                                                          Datatype::encode(args.datatype1), Datatype::encode(args.datatype2)));
 
-  Colls::gather(send, send_size, MPI_CURRENT_TYPE, recv, recv_size, MPI_CURRENT_TYPE2, root, MPI_COMM_WORLD);
+    if (name == "gather")
+      Colls::gather(send_buffer(args.send_size * args.datatype1->size()), args.send_size, args.datatype1,
+                 (rank == args.root) ? recv_buffer(args.recv_size * args.comm_size * args.datatype2->size()) : nullptr, args.recv_size, args.datatype2, args.root, MPI_COMM_WORLD);
+    else
+      Colls::allgather(send_buffer(args.send_size * args.datatype1->size()), args.send_size, args.datatype1,
+                       recv_buffer(args.recv_size * args.datatype2->size()), args.recv_size, args.datatype2, MPI_COMM_WORLD);
 
-  TRACE_smpi_comm_out(Actor::self()->getPid());
-  log_timed_action (action, clock);
-}
+    TRACE_smpi_comm_out(my_proc_id);
+  }
+};
+} // Replay Namespace
 
 static void action_scatter(simgrid::xbt::ReplayAction& action)
 {
@@ -619,41 +672,6 @@ static void action_reducescatter(simgrid::xbt::ReplayAction& action)
   log_timed_action (action, clock);
 }
 
-static void action_allgather(simgrid::xbt::ReplayAction& action)
-{
-  /* The structure of the allgather action for the rank 0 (total 4 processes) is the following:
-        0 allGather 275427 275427
-    where:
-        1) 275427 is the sendcount
-        2) 275427 is the recvcount
-        3) No more values mean that the datatype for sent and receive buffer is the default one, see
-    simgrid::smpi::Datatype::decode().
-  */
-  double clock = smpi_process()->simulated_elapsed();
-
-  CHECK_ACTION_PARAMS(action, 2, 2)
-  int sendcount = std::stoi(action[2]);
-  int recvcount = std::stoi(action[3]);
-
-  MPI_Datatype MPI_CURRENT_TYPE{(action.size() > 5) ? simgrid::smpi::Datatype::decode(action[4]) : MPI_DEFAULT_TYPE};
-  MPI_Datatype MPI_CURRENT_TYPE2{(action.size() > 5) ? simgrid::smpi::Datatype::decode(action[5]) : MPI_DEFAULT_TYPE};
-
-  void *sendbuf = smpi_get_tmp_sendbuffer(sendcount* MPI_CURRENT_TYPE->size());
-  void *recvbuf = smpi_get_tmp_recvbuffer(recvcount* MPI_CURRENT_TYPE2->size());
-
-  int my_proc_id = Actor::self()->getPid();
-
-  TRACE_smpi_comm_in(my_proc_id, __FUNCTION__,
-                     new simgrid::instr::CollTIData("allGather", -1, -1.0, sendcount, recvcount,
-                                                    Datatype::encode(MPI_CURRENT_TYPE),
-                                                    Datatype::encode(MPI_CURRENT_TYPE2)));
-
-  Colls::allgather(sendbuf, sendcount, MPI_CURRENT_TYPE, recvbuf, recvcount, MPI_CURRENT_TYPE2, MPI_COMM_WORLD);
-
-  TRACE_smpi_comm_out(my_proc_id);
-  log_timed_action (action, clock);
-}
-
 static void action_allgatherv(simgrid::xbt::ReplayAction& action)
 {
   /* The structure of the allgatherv action for the rank 0 (total 4 processes) is the following:
@@ -793,17 +811,17 @@ void smpi_replay_init(int* argc, char*** argv)
   xbt_replay_action_register("test",  [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::TestAction().execute(action); });
   xbt_replay_action_register("wait",  [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::WaitAction().execute(action); });
   xbt_replay_action_register("waitAll", [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::WaitAllAction().execute(action); });
-  xbt_replay_action_register("barrier",    simgrid::smpi::action_barrier);
-  xbt_replay_action_register("bcast",      simgrid::smpi::action_bcast);
-  xbt_replay_action_register("reduce",     simgrid::smpi::action_reduce);
-  xbt_replay_action_register("allReduce",  simgrid::smpi::action_allReduce);
-  xbt_replay_action_register("allToAll",   simgrid::smpi::action_allToAll);
+  xbt_replay_action_register("barrier", [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::BarrierAction().execute(action); });
+  xbt_replay_action_register("bcast",   [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::BcastAction().execute(action); });
+  xbt_replay_action_register("reduce",  [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::ReduceAction().execute(action); });
+  xbt_replay_action_register("allReduce", [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::AllReduceAction().execute(action); });
+  xbt_replay_action_register("allToAll", [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::AllToAllAction().execute(action); });
   xbt_replay_action_register("allToAllV",  simgrid::smpi::action_allToAllv);
-  xbt_replay_action_register("gather",     simgrid::smpi::action_gather);
+  xbt_replay_action_register("gather",   [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::GatherAction("gather").execute(action); });
   xbt_replay_action_register("scatter", simgrid::smpi::action_scatter);
   xbt_replay_action_register("gatherV",    simgrid::smpi::action_gatherv);
   xbt_replay_action_register("scatterV", simgrid::smpi::action_scatterv);
-  xbt_replay_action_register("allGather",  simgrid::smpi::action_allgather);
+  xbt_replay_action_register("allGather", [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::GatherAction("allGather").execute(action); });
   xbt_replay_action_register("allGatherV", simgrid::smpi::action_allgatherv);
   xbt_replay_action_register("reduceScatter",  simgrid::smpi::action_reducescatter);
   xbt_replay_action_register("compute", [](simgrid::xbt::ReplayAction& action) { simgrid::smpi::Replay::ComputeAction().execute(action); });