Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
Merge branch 'simgrid-udpor-integration' into 'master'
authorArnaud Giersch <arnaud.giersch@univ-fcomte.fr>
Thu, 23 Feb 2023 17:00:25 +0000 (18:00 +0100)
committerArnaud Giersch <arnaud.giersch@univ-fcomte.fr>
Thu, 23 Feb 2023 20:07:22 +0000 (21:07 +0100)
Phase 1 of Integration of UDPOR: Adding UDPOR Constructs

See merge request simgrid/simgrid!132

67 files changed:
ChangeLog
MANIFEST.in
examples/cpp/synchro-condition-variable-waituntil/s4u-synchro-condition-variable-waituntil.cpp
examples/cpp/synchro-condition-variable/s4u-synchro-condition-variable.cpp
examples/cpp/synchro-mutex/s4u-synchro-mutex.cpp
examples/cpp/synchro-mutex/s4u-synchro-mutex.tesh
examples/python/app-masterworkers/app-masterworkers.py
examples/python/comm-failure/comm-failure.py
examples/python/comm-suspend/comm-suspend.py
examples/python/platform-failures/platform-failures.py
examples/python/synchro-barrier/synchro-barrier.py
examples/python/synchro-mutex/synchro-mutex.py
examples/sthread/CMakeLists.txt
examples/sthread/pthread-mc-producer-consumer.tesh [new file with mode: 0644]
examples/sthread/pthread-mutex-simple.c
examples/sthread/pthread-mutex-simpledeadlock.c
examples/sthread/pthread-producer-consumer.c [new file with mode: 0644]
examples/sthread/pthread-producer-consumer.tesh [new file with mode: 0644]
examples/sthread/sthread-mutex-simple.c
include/simgrid/Exception.hpp
include/simgrid/actor.h
include/simgrid/engine.h
include/simgrid/plugins/ProducerConsumer.hpp
include/simgrid/s4u/Engine.hpp
include/simgrid/version.h.in
include/xbt.h
include/xbt/ex.h
include/xbt/module.h
include/xbt/sysdep.h
include/xbt/virtu.h
src/instr/instr_paje_header.cpp
src/kernel/EngineImpl.cpp
src/kernel/EngineImpl.hpp
src/kernel/context/ContextThread.cpp
src/kernel/resource/profile/StochasticDatedValue.cpp
src/mc/api/RemoteApp.cpp
src/mc/remote/AppSide.cpp
src/mc/remote/mc_protocol.h
src/s4u/s4u_Actor.cpp
src/s4u/s4u_Engine.cpp
src/simgrid/sg_config.cpp
src/smpi/internals/smpi_memory.cpp
src/smpi/mpi/smpi_file.cpp
src/smpi/mpi/smpi_request.cpp
src/smpi/mpi/smpi_win.cpp
src/sthread/sthread.c
src/sthread/sthread.h
src/sthread/sthread_impl.cpp
src/xbt/OsSemaphore.hpp
src/xbt/backtrace.cpp
src/xbt/dict.cpp
src/xbt/exception.cpp
src/xbt/log.cpp
src/xbt/parmap.hpp
src/xbt/xbt_log_layout_format.cpp
src/xbt/xbt_log_layout_simple.cpp
src/xbt/xbt_main.cpp [deleted file]
src/xbt/xbt_misc.cpp [new file with mode: 0644]
src/xbt/xbt_modinter.h [deleted file]
src/xbt/xbt_virtu.cpp [deleted file]
teshsuite/mc/mutex-handling/mutex-handling.cpp
teshsuite/models/lmm_usage/lmm_usage.cpp
teshsuite/models/maxmin_bench/maxmin_bench.cpp
teshsuite/s4u/CMakeLists.txt
teshsuite/s4u/monkey-masterworkers/monkey-masterworkers.py
teshsuite/xbt/mmalloc/mmalloc_test.cpp
tools/cmake/DefinePackages.cmake

index 73cd894..8776e69 100644 (file)
--- a/ChangeLog
+++ b/ChangeLog
@@ -43,6 +43,11 @@ Models:
 
 sthread:
  - Implement pthread_join in MC mode.
+ - Implement semaphore functions in sthread.
+
+XBT:
+ - simgrid::xbt::cmdline and simgrid::xbt::binary_name are gone.
+   Please use simgrid::s4u::Engine::get_cmdline() instead.
 
 Documentation:
  - New section in the user guide on the provided performance models.
index 680a6f0..89bf7ed 100644 (file)
@@ -597,9 +597,12 @@ include examples/smpi/trace_simple/trace_simple.c
 include examples/smpi/trace_simple/trace_simple.tesh
 include examples/sthread/pthread-mc-mutex-simple.tesh
 include examples/sthread/pthread-mc-mutex-simpledeadlock.tesh
+include examples/sthread/pthread-mc-producer-consumer.tesh
 include examples/sthread/pthread-mutex-simple.c
 include examples/sthread/pthread-mutex-simple.tesh
 include examples/sthread/pthread-mutex-simpledeadlock.c
+include examples/sthread/pthread-producer-consumer.c
+include examples/sthread/pthread-producer-consumer.tesh
 include examples/sthread/sthread-mutex-simple.c
 include examples/sthread/sthread-mutex-simple.tesh
 include teshsuite/catch_simgrid.hpp
@@ -2499,15 +2502,13 @@ include src/xbt/unit-tests_main.cpp
 include src/xbt/xbt_log_appender_file.cpp
 include src/xbt/xbt_log_layout_format.cpp
 include src/xbt/xbt_log_layout_simple.cpp
-include src/xbt/xbt_main.cpp
-include src/xbt/xbt_modinter.h
+include src/xbt/xbt_misc.cpp
 include src/xbt/xbt_os_file.cpp
 include src/xbt/xbt_os_time.c
 include src/xbt/xbt_parse_units.cpp
 include src/xbt/xbt_replay.cpp
 include src/xbt/xbt_str.cpp
 include src/xbt/xbt_str_test.cpp
-include src/xbt/xbt_virtu.cpp
 include teshsuite/kernel/CMakeLists.txt
 include teshsuite/mc/CMakeLists.txt
 include teshsuite/models/CMakeLists.txt
index 0edd2ab..67d1ac2 100644 (file)
@@ -3,7 +3,7 @@
 /* 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 <mutex>           /* std::mutex and std::lock_guard */
+#include <mutex>           /* std::mutex and std::scoped_lock */
 #include <simgrid/s4u.hpp> /* All of S4U */
 
 XBT_LOG_NEW_DEFAULT_CATEGORY(s4u_test, "a sample log category");
@@ -12,13 +12,12 @@ namespace sg4 = simgrid::s4u;
 static void competitor(int id, sg4::ConditionVariablePtr cv, sg4::MutexPtr mtx, std::shared_ptr<bool> ready)
 {
   XBT_INFO("Entering the race...");
-  std::unique_lock lck(*mtx);
+  std::unique_lock lock(*mtx);
   while (not *ready) {
     auto now = sg4::Engine::get_clock();
-    if (cv->wait_until(lck, now + (id+1)*0.25) == std::cv_status::timeout) {
+    if (cv->wait_until(lock, now + (id + 1) * 0.25) == std::cv_status::timeout) {
       XBT_INFO("Out of wait_until (timeout)");
-    }
-    else {
+    } else {
       XBT_INFO("Out of wait_until (YAY!)");
     }
   }
@@ -29,7 +28,7 @@ static void go(sg4::ConditionVariablePtr cv, sg4::MutexPtr mtx, std::shared_ptr<
 {
   XBT_INFO("Are you ready? ...");
   sg4::this_actor::sleep_for(3);
-  std::unique_lock lck(*mtx);
+  const std::scoped_lock lock(*mtx);
   XBT_INFO("Go go go!");
   *ready = true;
   cv->notify_all();
index cf1a1db..b471b65 100644 (file)
@@ -3,7 +3,7 @@
 /* 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 <mutex>           /* std::mutex and std::lock_guard */
+#include <mutex>           /* std::mutex and std::scoped_lock */
 #include <simgrid/s4u.hpp> /* All of S4U */
 
 XBT_LOG_NEW_DEFAULT_CATEGORY(s4u_test, "a sample log category");
@@ -11,7 +11,7 @@ namespace sg4 = simgrid::s4u;
 
 static void worker_fun(sg4::ConditionVariablePtr cv, sg4::MutexPtr mutex, std::string& data, bool& done)
 {
-  std::unique_lock lock(*mutex);
+  const std::scoped_lock lock(*mutex);
 
   XBT_INFO("Start processing data which is '%s'.", data.c_str());
   data += " after processing";
@@ -34,7 +34,7 @@ static void master_fun()
                                    std::ref(done));
 
   // wait for the worker
-  cv->wait(std::unique_lock<sg4::Mutex>(*mutex), [&done]() { return done; });
+  cv->wait(std::unique_lock(*mutex), [&done]() { return done; });
   XBT_INFO("data is now '%s'.", data.c_str());
 
   worker->join();
index 429fccd..bab6731 100644 (file)
@@ -5,7 +5,7 @@
 
 #include "simgrid/s4u.hpp" /* All of S4U */
 #include "xbt/config.hpp"
-#include <mutex> /* std::mutex and std::lock_guard */
+#include <mutex> /* std::mutex and std::scoped_lock */
 
 namespace sg4 = simgrid::s4u;
 
@@ -29,14 +29,14 @@ static void worker(sg4::MutexPtr mutex, int& result)
   mutex->unlock();
 }
 
-static void workerLockGuard(sg4::MutexPtr mutex, int& result)
+static void workerScopedLock(sg4::MutexPtr mutex, int& result)
 {
-  // Simply use the std::lock_guard like this
+  // Simply use the std::scoped_lock like this
   // It's like a lock() that would do the unlock() automatically when getting out of scope
-  std::lock_guard lock(*mutex);
+  const std::scoped_lock lock(*mutex);
 
   // then you are in a safe zone
-  XBT_INFO("Hello s4u, I'm ready to compute after a lock_guard");
+  XBT_INFO("Hello s4u, I'm ready to compute after a scoped_lock");
   // update the results
   result += 1;
   XBT_INFO("I'm done, good bye");
@@ -54,7 +54,7 @@ int main(int argc, char** argv)
 
   for (int i = 0; i < cfg_actor_count; i++) {
     sg4::MutexPtr mutex = sg4::Mutex::create();
-    sg4::Actor::create("worker", sg4::Host::by_name("Jupiter"), workerLockGuard, mutex, std::ref(result[i]));
+    sg4::Actor::create("worker", sg4::Host::by_name("Jupiter"), workerScopedLock, mutex, std::ref(result[i]));
     sg4::Actor::create("worker", sg4::Host::by_name("Tremblay"), worker, mutex, std::ref(result[i]));
   }
 
index fbec803..ca6b489 100644 (file)
@@ -1,17 +1,17 @@
 #!/usr/bin/env tesh
 
 $ ${bindir:=.}/s4u-synchro-mutex
-> [Jupiter:worker:(1) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a lock_guard
+> [Jupiter:worker:(1) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a scoped_lock
 > [Jupiter:worker:(1) 0.000000] [s4u_test/INFO] I'm done, good bye
-> [Jupiter:worker:(3) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a lock_guard
+> [Jupiter:worker:(3) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a scoped_lock
 > [Jupiter:worker:(3) 0.000000] [s4u_test/INFO] I'm done, good bye
-> [Jupiter:worker:(5) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a lock_guard
+> [Jupiter:worker:(5) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a scoped_lock
 > [Jupiter:worker:(5) 0.000000] [s4u_test/INFO] I'm done, good bye
-> [Jupiter:worker:(7) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a lock_guard
+> [Jupiter:worker:(7) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a scoped_lock
 > [Jupiter:worker:(7) 0.000000] [s4u_test/INFO] I'm done, good bye
-> [Jupiter:worker:(9) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a lock_guard
+> [Jupiter:worker:(9) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a scoped_lock
 > [Jupiter:worker:(9) 0.000000] [s4u_test/INFO] I'm done, good bye
-> [Jupiter:worker:(11) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a lock_guard
+> [Jupiter:worker:(11) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a scoped_lock
 > [Jupiter:worker:(11) 0.000000] [s4u_test/INFO] I'm done, good bye
 > [Tremblay:worker:(2) 0.000000] [s4u_test/INFO] Hello s4u, I'm ready to compute after a regular lock
 > [Tremblay:worker:(2) 0.000000] [s4u_test/INFO] I'm done, good bye
index 4c146f2..856d9e3 100644 (file)
@@ -56,7 +56,7 @@ def worker(*args):
 
 # main-begin
 if __name__ == '__main__':
-    assert len(sys.argv) > 2, f"Usage: python app-masterworkers.py platform_file deployment_file"
+    assert len(sys.argv) > 2, "Usage: python app-masterworkers.py platform_file deployment_file"
 
     e = Engine(sys.argv)
 
index 91ec73d..4b70b33 100644 (file)
@@ -18,14 +18,14 @@ def sender(mailbox1_name: str, mailbox2_name: str) -> None:
     this_actor.info(f"Initiating asynchronous send to {mailbox2.name}")
     comm2: Comm = mailbox2.put_async(666, 2)
 
-    this_actor.info(f"Calling wait_any..")
+    this_actor.info("Calling wait_any..")
     pending_comms = [comm1, comm2]
     try:
         index = Comm.wait_any([comm1, comm2])
         this_actor.info(f"Wait any returned index {index} (comm to {pending_comms[index].mailbox.name})")
     except NetworkFailureException:
-        this_actor.info(f"Sender has experienced a network failure exception, so it knows that something went wrong")
-        this_actor.info(f"Now it needs to figure out which of the two comms failed by looking at their state:")
+        this_actor.info("Sender has experienced a network failure exception, so it knows that something went wrong")
+        this_actor.info("Now it needs to figure out which of the two comms failed by looking at their state:")
 
     this_actor.info(f"  Comm to {comm1.mailbox.name} has state: {comm1.state_str}")
     this_actor.info(f"  Comm to {comm2.mailbox.name} has state: {comm2.state_str}")
index badf091..85646ed 100644 (file)
@@ -43,7 +43,7 @@ def sender():
     comm.resume()
     comm.wait()
     this_actor.info(f"There is {comm.remaining:.0f} bytes to transfer after the communication completion.")
-    this_actor.info(f"Suspending a completed activity is a no-op.")
+    this_actor.info("Suspending a completed activity is a no-op.")
     comm.suspend()
 
 
index 5176658..9850ab4 100644 (file)
@@ -82,7 +82,7 @@ def sleeper():
     this_actor.info("done sleeping.")
 
 if __name__ == '__main__':
-    assert len(sys.argv) > 2, f"Usage: python app-masterworkers.py platform_file deployment_file"
+    assert len(sys.argv) > 2, "Usage: python app-masterworkers.py platform_file deployment_file"
 
     e = Engine(sys.argv)
 
index f1f6fae..6b81a1d 100644 (file)
@@ -30,7 +30,7 @@ def worker(barrier: Barrier):
     """ Wait on the barrier and exits.
     :param barrier: Barrier to be awaited
     """
-    this_actor.info(f"Waiting on the barrier")
+    this_actor.info("Waiting on the barrier")
     barrier.wait()
     this_actor.info("Bye")
 
@@ -45,7 +45,7 @@ def master(actor_count: int):
     this_actor.info(f"Spawning {workers_count} workers")
     for i in range(workers_count):
         Actor.create(f"worker-{i}", Host.by_name("Jupiter"), worker, barrier)
-    this_actor.info(f"Waiting on the barrier")
+    this_actor.info("Waiting on the barrier")
     barrier.wait()
     this_actor.info("Bye")
 
index 298de0b..1a202ba 100644 (file)
@@ -35,9 +35,9 @@ class ResultHolder:
 def worker_context_manager(mutex: Mutex, result: ResultHolder):
     # When using a context manager, the lock and the unlock are automatic. This is the easiest approach
     with mutex:
-        this_actor.info(f"Hello simgrid, I'm ready to compute after acquiring the mutex from a context manager")
+        this_actor.info("Hello simgrid, I'm ready to compute after acquiring the mutex from a context manager")
         result.value += 1
-    this_actor.info(f"I'm done, good bye")
+    this_actor.info("I'm done, good bye")
 
 
 def worker(mutex: Mutex, result: ResultHolder):
index a59d61f..d53f419 100644 (file)
@@ -5,7 +5,8 @@ find_package(Threads REQUIRED)
 #########################################################################
 
 foreach(x
-        mutex-simple)
+        mutex-simple
+       producer-consumer)
 
   if("${CMAKE_SYSTEM}" MATCHES "Linux")
     add_executable       (pthread-${x} EXCLUDE_FROM_ALL pthread-${x}.c)
diff --git a/examples/sthread/pthread-mc-producer-consumer.tesh b/examples/sthread/pthread-mc-producer-consumer.tesh
new file mode 100644 (file)
index 0000000..c271417
--- /dev/null
@@ -0,0 +1,7 @@
+# We ignore the LD_PRELOAD lines from the expected output because they contain the build path
+! ignore .*LD_PRELOAD.*
+
+$ ${bindir:=.}/../../bin/simgrid-mc --cfg=model-check/setenv:LD_PRELOAD=${libdir:=.}/libsgmalloc.so:${libdir:=.}/libsthread.so ${bindir:=.}/pthread-producer-consumer -q
+> [0.000000] [sthread/INFO] Starting the simulation.
+> [0.000000] [mc_dfs/INFO] Start a DFS exploration. Reduction is: dpor.
+> [0.000000] [mc_dfs/INFO] DFS exploration ended. 1819 unique states visited; 108 backtracks (6808 transition replays, 4882 states visited overall)
\ No newline at end of file
index d97bfe8..1d39141 100644 (file)
@@ -21,8 +21,8 @@ int main(int argc, char* argv[])
   int id[2] = {0, 1};
   pthread_t thread1;
   pthread_t thread2;
-  pthread_create(&thread1, NULL, thread_fun, (void*)&id[0]);
-  pthread_create(&thread2, NULL, thread_fun, (void*)&id[1]);
+  pthread_create(&thread1, NULL, thread_fun, &id[0]);
+  pthread_create(&thread2, NULL, thread_fun, &id[1]);
   fprintf(stderr, "All threads are started.\n");
   pthread_join(thread1, NULL);
   pthread_join(thread2, NULL);
index 433852a..09be6c1 100644 (file)
@@ -39,8 +39,8 @@ int main(int argc, char* argv[])
   int id[2] = {0, 1};
   pthread_t thread1;
   pthread_t thread2;
-  pthread_create(&thread1, NULL, thread_fun1, (void*)&id[0]);
-  pthread_create(&thread2, NULL, thread_fun2, (void*)&id[1]);
+  pthread_create(&thread1, NULL, thread_fun1, &id[0]);
+  pthread_create(&thread2, NULL, thread_fun2, &id[1]);
   fprintf(stderr, "All threads are started.\n");
   pthread_join(thread1, NULL);
   pthread_join(thread2, NULL);
diff --git a/examples/sthread/pthread-producer-consumer.c b/examples/sthread/pthread-producer-consumer.c
new file mode 100644 (file)
index 0000000..8fd1a0b
--- /dev/null
@@ -0,0 +1,84 @@
+/* Copyright (c) 2002-2023. 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. */
+
+/* Simple producer/consumer example with pthreads and semaphores */
+
+#include <pthread.h>
+#include <semaphore.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#define AmountProduced 3 /* Amount of items produced by a producer */
+#define AmountConsumed 3 /* Amount of items consumed by a consumer */
+#define ProducerCount 2  /* Amount of producer threads*/
+#define ConsumerCount 2  /* Amount of consumer threads*/
+#define BufferSize 4     /* Size of the buffer */
+
+sem_t empty;
+sem_t full;
+int in  = 0;
+int out = 0;
+int buffer[BufferSize];
+pthread_mutex_t mutex;
+int do_output = 1;
+
+static void* producer(void* id)
+{
+  for (int i = 0; i < AmountProduced; i++) {
+    sem_wait(&empty);
+    pthread_mutex_lock(&mutex);
+    buffer[in] = i;
+    if (do_output)
+      fprintf(stderr, "Producer %d: Insert Item %d at %d\n", *((int*)id), buffer[in], in);
+    in = (in + 1) % BufferSize;
+    pthread_mutex_unlock(&mutex);
+    sem_post(&full);
+  }
+  return NULL;
+}
+static void* consumer(void* id)
+{
+  for (int i = 0; i < AmountConsumed; i++) {
+    sem_wait(&full);
+    pthread_mutex_lock(&mutex);
+    int item = buffer[out];
+    if (do_output)
+      fprintf(stderr, "Consumer %d: Remove Item %d from %d\n", *((int*)id), item, out);
+    out = (out + 1) % BufferSize;
+    pthread_mutex_unlock(&mutex);
+    sem_post(&empty);
+  }
+  return NULL;
+}
+
+int main(int argc, char** argv)
+{
+  if (argc == 2 && strcmp(argv[1], "-q") == 0)
+    do_output = 0;
+  pthread_t pro[2];
+  pthread_t con[2];
+  pthread_mutex_init(&mutex, NULL);
+  sem_init(&empty, 0, BufferSize);
+  sem_init(&full, 0, 0);
+
+  int ids[10] = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; // The identity of each thread (for debug messages)
+
+  for (int i = 0; i < ProducerCount; i++)
+    pthread_create(&pro[i], NULL, producer, &ids[i]);
+  for (int i = 0; i < ConsumerCount; i++)
+    pthread_create(&con[i], NULL, consumer, &ids[i]);
+
+  for (int i = 0; i < ProducerCount; i++)
+    pthread_join(pro[i], NULL);
+  for (int i = 0; i < ConsumerCount; i++)
+    pthread_join(con[i], NULL);
+
+  pthread_mutex_destroy(&mutex);
+  sem_destroy(&empty);
+  sem_destroy(&full);
+
+  return 0;
+}
diff --git a/examples/sthread/pthread-producer-consumer.tesh b/examples/sthread/pthread-producer-consumer.tesh
new file mode 100644 (file)
index 0000000..0c035f0
--- /dev/null
@@ -0,0 +1,15 @@
+$ env ASAN_OPTIONS=verify_asan_link_order=0:$ASAN_OPTIONS LD_PRELOAD=${libdir:=.}/libsthread.so ./pthread-producer-consumer
+> [0.000000] [sthread/INFO] Starting the simulation.
+> Producer 1: Insert Item 0 at 0
+> Producer 2: Insert Item 0 at 1
+> Consumer 1: Remove Item 0 from 0
+> Producer 1: Insert Item 1 at 2
+> Consumer 2: Remove Item 0 from 1
+> Producer 2: Insert Item 1 at 3
+> Consumer 1: Remove Item 1 from 2
+> Producer 1: Insert Item 2 at 0
+> Consumer 2: Remove Item 1 from 3
+> Producer 2: Insert Item 2 at 1
+> Consumer 1: Remove Item 2 from 0
+> Consumer 2: Remove Item 2 from 1
+> [0.000000] [sthread/INFO] All threads exited. Terminating the simulation.
index 76fa009..dccec54 100644 (file)
@@ -21,8 +21,8 @@ int main(int argc, char* argv[])
   int id[2] = {0, 1};
   sthread_t thread1;
   sthread_t thread2;
-  sthread_create(&thread1, NULL, thread_fun, (void*)&id[0]);
-  sthread_create(&thread2, NULL, thread_fun, (void*)&id[1]);
+  sthread_create(&thread1, NULL, thread_fun, &id[0]);
+  sthread_create(&thread2, NULL, thread_fun, &id[1]);
   fprintf(stderr, "All threads are started.\n");
   sthread_join(thread1, NULL);
   sthread_join(thread2, NULL);
index e580ea9..ca94f51 100644 (file)
@@ -3,11 +3,12 @@
 /* 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. */
 
-/* This file defines all possible exception that could occur in a SimGrid library. */
+/* This file defines all possible exceptions that could occur in a SimGrid library. */
 
 #ifndef SIMGRID_EXCEPTIONS_HPP
 #define SIMGRID_EXCEPTIONS_HPP
 
+#include <simgrid/actor.h>
 #include <xbt/backtrace.hpp>
 #include <xbt/ex.h>
 #include <xbt/string.hpp>
@@ -31,7 +32,7 @@ class ThrowPoint {
 public:
   ThrowPoint() = default;
   explicit ThrowPoint(const char* file, int line, const char* function, Backtrace&& bt, std::string&& actor_name,
-                      int pid)
+                      aid_t pid)
       : file_(file)
       , line_(line)
       , function_(function)
@@ -46,12 +47,13 @@ public:
   const char* function_ = nullptr;
   Backtrace backtrace_;
   std::string procname_ = ""; /**< Name of the process who thrown this */
-  int pid_              = 0;  /**< PID of the process who thrown this */
+  aid_t pid_            = 0;  /**< PID of the process who thrown this */
 };
 
 /** Create a ThrowPoint with (__FILE__, __LINE__, __func__) */
 #define XBT_THROW_POINT                                                                                                \
-  ::simgrid::xbt::ThrowPoint(__FILE__, __LINE__, __func__, simgrid::xbt::Backtrace(), xbt_procname(), xbt_getpid())
+  ::simgrid::xbt::ThrowPoint(__FILE__, __LINE__, __func__, simgrid::xbt::Backtrace(), sg_actor_self_get_name(),        \
+                             sg_actor_self_get_pid())
 
 class XBT_PUBLIC ImpossibleError : public std::logic_error {
 public:
index dcee507..fb34364 100644 (file)
@@ -72,6 +72,7 @@ XBT_PUBLIC void sg_actor_detach();
 XBT_PUBLIC sg_actor_t sg_actor_self();
 XBT_PUBLIC aid_t sg_actor_self_get_pid();
 XBT_PUBLIC aid_t sg_actor_self_get_ppid();
+/** Returns the name of the current actor (or "maestro" if maestro is running) */
 XBT_PUBLIC const char* sg_actor_self_get_name();
 XBT_PUBLIC void* sg_actor_self_get_data();
 XBT_PUBLIC void sg_actor_self_set_data(void* data);
index b4e8c08..d3ff02b 100644 (file)
@@ -11,9 +11,9 @@
 
 SG_BEGIN_DECL /* C interface */
 
-    /** Initialize the SimGrid engine, taking the command line parameters of your main function. */
-    XBT_PUBLIC void
-    simgrid_init(int* argc, char** argv);
+/** Initialize the SimGrid engine, taking the command line parameters of your main function. */
+XBT_PUBLIC void
+simgrid_init(int* argc, char** argv);
 
 /** Creates a new platform, including hosts, links, and the routing table.
  *
index 1ecaa1e..55dc6d6 100644 (file)
@@ -104,7 +104,7 @@ public:
    */
   ProducerConsumer* set_max_queue_size(unsigned int max_queue_size)
   {
-    std::unique_lock<s4u::Mutex> lock(*mutex_);
+    const std::lock_guard<s4u::Mutex> lock(*mutex_);
     max_queue_size_ = max_queue_size;
     return this;
   }
index 94b0202..c9fd475 100644 (file)
@@ -56,6 +56,7 @@ public:
   static s4u::Engine* get_instance();
   static s4u::Engine* get_instance(int* argc, char** argv);
   static bool has_instance() { return instance_ != nullptr; }
+  const std::vector<std::string>& get_cmdline() const;
 
   /**
    * Creates a new platform, including hosts, links, and the routing table.
@@ -199,7 +200,10 @@ public:
     return res;
   }
 
-  kernel::EngineImpl* get_impl() const { return pimpl; }
+  kernel::EngineImpl* get_impl() const
+  {
+    return pimpl_;
+  }
 
   /** Returns whether SimGrid was initialized yet -- mostly for internal use */
   static bool is_initialized();
@@ -255,7 +259,7 @@ private:
   static xbt::signal<void(void)> on_deadlock;
   static xbt::signal<void()> on_simulation_end;
 
-  kernel::EngineImpl* const pimpl;
+  kernel::EngineImpl* const pimpl_;
   static Engine* instance_;
   void initialize(int* argc, char** argv);
 };
index acd8644..daa8e9c 100644 (file)
@@ -8,6 +8,8 @@
 #ifndef SIMGRID_VERSION_H
 #define SIMGRID_VERSION_H
 
+#include "xbt/base.h"
+
 #define SIMGRID_GIT_VERSION   "@GIT_VERSION@"
 
 /** Define the version numbers of the used header files.
index deafed8..0f6e5b8 100644 (file)
@@ -17,8 +17,6 @@
 #include <xbt/asserts.h>
 #include <xbt/log.h>
 
-#include <xbt/module.h>
-
 #include <xbt/dynar.h>
 #include <xbt/dict.h>
 
index 83e90c3..f221afe 100644 (file)
@@ -11,7 +11,6 @@
 #include <xbt/base.h>
 #include <xbt/misc.h>
 #include <xbt/sysdep.h>
-#include <xbt/virtu.h>
 
 /** @addtogroup XBT_ex_c
  *  @brief Exceptions support (C)
index 43fe15d..ee9d548 100644 (file)
@@ -8,11 +8,20 @@
 #ifndef XBT_MODULE_H
 #define XBT_MODULE_H
 
-#include <xbt/misc.h>           /* XBT_PUBLIC */
+// avoid deprecation warning on include (remove entire file with XBT_ATTRIB_DEPRECATED_v337)
+#ifndef XBT_MODULE_H_NO_DEPRECATED_WARNING
+#warning xbt/module.h is deprecated and will be removed in v3.37.
+#endif
+
+#include <simgrid/engine.h>
+#include <xbt/base.h>
 
 SG_BEGIN_DECL
 
-XBT_PUBLIC void xbt_init(int* argc, char** argv);
+XBT_ATTRIB_DEPRECATED_v337("Please use simgrid_init(&argc, argv) instead") static void xbt_init(int* argc, char** argv)
+{
+  simgrid_init(argc, argv);
+}
 
 SG_END_DECL
 
index a4800a1..e78e0bd 100644 (file)
@@ -78,8 +78,7 @@ static XBT_ALWAYS_INLINE void *xbt_realloc(void *p, size_t s) {
   return res;
 }
 
-/** @brief like free
-    @hideinitializer */
+/** @brief like free */
 #define xbt_free(p) free(p) /*nothing specific to do here. A poor valgrind replacement? */
 
 #ifdef __cplusplus
index b1a6e04..0789a9a 100644 (file)
@@ -8,29 +8,25 @@
 #ifndef XBT_VIRTU_H
 #define XBT_VIRTU_H
 
+#include <simgrid/actor.h>
 #include <xbt/base.h>
 
-#ifdef __cplusplus
-#include <string>
-#include <vector>
-
-namespace simgrid {
-namespace xbt {
-
-/* Get the name of the UNIX process englobing the world */
-XBT_PUBLIC_DATA std::string binary_name;
-/** Contains all the parameters we got from the command line (including argv[0]) */
-XBT_PUBLIC_DATA std::vector<std::string> cmdline;
-
-} // namespace xbt
-} // namespace simgrid
+// avoid deprecation warning on include (remove entire file with XBT_ATTRIB_DEPRECATED_v337)
+#ifndef XBT_VIRTU_H_NO_DEPRECATED_WARNING
+#warning xbt/virtu.h is deprecated and will be removed in v3.37.
 #endif
 
 SG_BEGIN_DECL
 
-XBT_PUBLIC const char* xbt_procname(void);
+XBT_ATTRIB_DEPRECATED_v337("Please use sg_actor_self_get_name()") static const char* xbt_procname(void)
+{
+  return sg_actor_self_get_name();
+}
 
-XBT_PUBLIC int xbt_getpid(void);
+XBT_ATTRIB_DEPRECATED_v337("Please use sg_actor_self_get_pid()") static int xbt_getpid(void)
+{
+  return sg_actor_self_get_pid();
+};
 
 SG_END_DECL
 
index 487f605..d69e4dd 100644 (file)
@@ -1,14 +1,13 @@
-/* Copyright (c) 2010-2023. The SimGrid Team.
- * All rights reserved.                                                     */
+/* Copyright (c) 2010-2023. 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 "simgrid/Exception.hpp"
 #include "simgrid/version.h"
 #include "src/instr/instr_private.hpp"
 #include "src/smpi/include/private.hpp"
-#include "xbt/virtu.h" /* xbt::cmdline */
+#include <simgrid/Exception.hpp>
+#include <simgrid/s4u/Engine.hpp>
 
 extern std::ofstream tracing_file;
 namespace simgrid::instr::paje {
@@ -18,7 +17,7 @@ void dump_generator_version()
   tracing_file << "#This file was generated using SimGrid-" << SIMGRID_VERSION_MAJOR << "." << SIMGRID_VERSION_MINOR
                << "." << SIMGRID_VERSION_PATCH << '\n';
   tracing_file << "#[";
-  for (auto const& str : simgrid::xbt::cmdline) {
+  for (auto const& str : simgrid::s4u::Engine::get_instance()->get_cmdline()) {
     tracing_file << str << " ";
   }
   tracing_file << "]\n";
index 1547836..d493956 100644 (file)
@@ -19,8 +19,8 @@
 #include "src/simgrid/math_utils.h"
 #include "src/simgrid/sg_config.hpp"
 #include "src/smpi/include/smpi_actor.hpp"
-#include "src/xbt/xbt_modinter.h" /* whether initialization was already done */
-#include "xbt/module.h"
+
+#include "xbt/log.hpp"
 
 #include <boost/algorithm/string/predicate.hpp>
 #include <dlfcn.h>
@@ -139,6 +139,9 @@ static void install_signal_handlers()
   }
 }
 
+static simgrid::config::Flag<bool> cfg_dbg_clean_atexit{
+    "debug/clean-atexit", "Whether to cleanup SimGrid at exit. Disable it if your code segfaults after its end.", true};
+
 namespace simgrid::kernel {
 
 EngineImpl::~EngineImpl()
@@ -175,8 +178,14 @@ void EngineImpl::initialize(int* argc, char** argv)
   simgrid::mc::AppSide::initialize();
 #endif
 
-  if (xbt_initialized == 0) {
-    xbt_init(argc, argv);
+  if (static bool inited = false; not inited) {
+    inited = true;
+    xbt_log_init(argc, argv);
+
+    simgrid::xbt::install_exception_handler();
+
+    for (int i = 0; i < *argc; i++)
+      cmdline_.emplace_back(argv[i]);
 
     sg_config_init(argc, argv);
   }
@@ -188,7 +197,7 @@ void EngineImpl::initialize(int* argc, char** argv)
   /* register a function to be called after the environment creation */
   s4u::Engine::on_platform_created_cb([this]() { this->presolve(); });
 
-  if (config::get_value<bool>("debug/clean-atexit"))
+  if (cfg_dbg_clean_atexit)
     atexit(shutdown);
 }
 
index 889f726..6862409 100644 (file)
@@ -57,6 +57,8 @@ class EngineImpl {
   std::unique_ptr<void, std::function<int(void*)>> platf_handle_; //!< handle for platform library
   friend s4u::Engine;
 
+  std::vector<std::string> cmdline_; // Copy of the argv we got (including argv[0])
+
 public:
   EngineImpl() = default;
 
@@ -69,6 +71,10 @@ public:
 #endif
 
   void initialize(int* argc, char** argv);
+  const std::vector<std::string>& get_cmdline() const
+  {
+    return cmdline_;
+  }
   void load_platform(const std::string& platf);
   void load_deployment(const std::string& file) const;
   void seal_platform() const;
@@ -97,8 +103,14 @@ public:
   const std::vector<resource::Model*>& get_all_models() const { return models_; }
 
   static bool has_instance() { return s4u::Engine::has_instance(); }
-  static EngineImpl* get_instance() { return s4u::Engine::get_instance()->pimpl; }
-  static EngineImpl* get_instance(int* argc, char** argv) { return s4u::Engine::get_instance(argc, argv)->pimpl; }
+  static EngineImpl* get_instance()
+  {
+    return s4u::Engine::get_instance()->pimpl_;
+  }
+  static EngineImpl* get_instance(int* argc, char** argv)
+  {
+    return s4u::Engine::get_instance(argc, argv)->pimpl_;
+  }
 
   actor::ActorCodeFactory get_function(const std::string& name)
   {
index 68bb417..40efb6b 100644 (file)
@@ -8,7 +8,6 @@
 #include "simgrid/Exception.hpp"
 #include "src/internal_config.h" /* loads context system definitions */
 #include "src/kernel/EngineImpl.hpp"
-#include "src/xbt/xbt_modinter.h" /* prototype of os thread module's init/exit in XBT */
 #include "xbt/function_types.h"
 
 #include <boost/core/demangle.hpp>
index 4f7208a..c1d383e 100644 (file)
@@ -4,8 +4,10 @@
  * under the terms of the license (GNU LGPL) which comes with this package. */
 
 #include "src/kernel/resource/profile/StochasticDatedValue.hpp"
-#include "xbt.h"
+
+#include "xbt/asserts.h"
 #include "xbt/random.hpp"
+
 #include <math.h>
 
 namespace simgrid::kernel::profile {
index ee1627f..9007e82 100644 (file)
@@ -171,13 +171,13 @@ void RemoteApp::get_actors_status(std::map<aid_t, ActorState>& whereto) const
   model_checker_->channel().send(MessageType::ACTORS_STATUS);
 
   s_mc_message_actors_status_answer_t answer;
-  ssize_t received = model_checker_->channel().receive(answer);
-  xbt_assert(received != -1, "Could not receive message");
-  xbt_assert(received == sizeof(answer) && answer.type == MessageType::ACTORS_STATUS_REPLY,
-             "Received unexpected message %s (%i, size=%i) "
-             "expected MessageType::ACTORS_STATUS_REPLY (%i, size=%i)",
-             to_c_str(answer.type), (int)answer.type, (int)received, (int)MessageType::ACTORS_STATUS_REPLY,
-             (int)sizeof(answer));
+  ssize_t answer_size = model_checker_->channel().receive(answer);
+  xbt_assert(answer_size != -1, "Could not receive message");
+  xbt_assert(answer_size == sizeof answer && answer.type == MessageType::ACTORS_STATUS_REPLY,
+             "Received unexpected message %s (%i, size=%zd) "
+             "expected MessageType::ACTORS_STATUS_REPLY (%i, size=%zu)",
+             to_c_str(answer.type), (int)answer.type, answer_size, (int)MessageType::ACTORS_STATUS_REPLY,
+             sizeof answer);
 
   // Message sanity checks
   xbt_assert(answer.count >= 0, "Received an ACTOR_STATUS_REPLY message with an actor count of '%d' < 0", answer.count);
@@ -206,17 +206,16 @@ void RemoteApp::get_actors_status(std::map<aid_t, ActorState>& whereto) const
   std::vector<s_mc_message_simcall_probe_one_t> probes(answer.transition_count);
   if (answer.transition_count > 0) {
     for (auto& probe : probes) {
-      size_t size      = sizeof(s_mc_message_simcall_probe_one_t);
-      ssize_t received = model_checker_->channel().receive(&probe, size);
+      ssize_t received = model_checker_->channel().receive(probe);
       xbt_assert(received >= 0, "Could not receive response to ACTORS_PROBE message (%s)", strerror(errno));
-      xbt_assert(static_cast<size_t>(received) == size,
+      xbt_assert(static_cast<size_t>(received) == sizeof probe,
                  "Could not receive response to ACTORS_PROBE message (%zd bytes received != %zu bytes expected",
-                 received, size);
+                 received, sizeof probe);
     }
   }
 
   whereto.clear();
-  auto probes_iter = std::move_iterator(probes.begin());
+  std::move_iterator probes_iter(probes.begin());
 
   for (const auto& actor : status) {
     xbt_assert(actor.n_transitions == 0 || actor.n_transitions == actor.max_considered,
@@ -225,14 +224,13 @@ void RemoteApp::get_actors_status(std::map<aid_t, ActorState>& whereto) const
                "(currently %d), but only %d transition(s) was/were said to be encoded",
                actor.max_considered, actor.n_transitions);
 
-    auto actor_transitions = std::vector<std::shared_ptr<Transition>>(actor.n_transitions);
+    std::vector<std::shared_ptr<Transition>> actor_transitions;
     for (int times_considered = 0; times_considered < actor.n_transitions; times_considered++, probes_iter++) {
       std::stringstream stream((*probes_iter).buffer.data());
-      auto transition = std::shared_ptr<Transition>(deserialize_transition(actor.aid, times_considered, stream));
-      actor_transitions[times_considered] = std::move(transition);
+      actor_transitions.emplace_back(deserialize_transition(actor.aid, times_considered, stream));
     }
 
-    XBT_DEBUG("Received %d transitions for actor %ld", actor.n_transitions, actor.aid);
+    XBT_DEBUG("Received %zu transitions for actor %ld", actor_transitions.size(), actor.aid);
     whereto.try_emplace(actor.aid, actor.aid, actor.enabled, actor.max_considered, std::move(actor_transitions));
   }
 }
@@ -241,13 +239,13 @@ void RemoteApp::check_deadlock() const
 {
   xbt_assert(model_checker_->channel().send(MessageType::DEADLOCK_CHECK) == 0, "Could not check deadlock state");
   s_mc_message_int_t message;
-  ssize_t s = model_checker_->channel().receive(message);
-  xbt_assert(s != -1, "Could not receive message");
-  xbt_assert(s == sizeof(message) && message.type == MessageType::DEADLOCK_CHECK_REPLY,
-             "Received unexpected message %s (%i, size=%i) "
-             "expected MessageType::DEADLOCK_CHECK_REPLY (%i, size=%i)",
-             to_c_str(message.type), (int)message.type, (int)s, (int)MessageType::DEADLOCK_CHECK_REPLY,
-             (int)sizeof(message));
+  ssize_t received = model_checker_->channel().receive(message);
+  xbt_assert(received != -1, "Could not receive message");
+  xbt_assert(received == sizeof message && message.type == MessageType::DEADLOCK_CHECK_REPLY,
+             "Received unexpected message %s (%i, size=%zd) "
+             "expected MessageType::DEADLOCK_CHECK_REPLY (%i, size=%zu)",
+             to_c_str(message.type), (int)message.type, received, (int)MessageType::DEADLOCK_CHECK_REPLY,
+             sizeof message);
 
   if (message.value != 0) {
     XBT_CINFO(mc_global, "Counter-example execution trace:");
index 9db7806..a32ece1 100644 (file)
@@ -217,10 +217,8 @@ void AppSide::handle_actors_status() const
     }
     XBT_DEBUG("Deliver ACTOR_TRANSITION_PROBE payload");
 
-    for (const auto& probe : probes) {
-      size_t size = sizeof(s_mc_message_simcall_probe_one_t);
-      xbt_assert(channel_.send(&probe, size) == 0, "Could not send ACTOR_TRANSITION_PROBE payload (%zu bytes)", size);
-    }
+    for (const auto& probe : probes)
+      xbt_assert(channel_.send(probe) == 0, "Could not send ACTOR_TRANSITION_PROBE payload");
   }
 }
 
index 8636b5c..435d60d 100644 (file)
@@ -118,8 +118,8 @@ struct s_mc_message_actors_status_one_t { // an array of `s_mc_message_actors_st
 };
 
 // Answer from an actor to the question "what are you about to run?"
-struct s_mc_message_simcall_probe_one_t { // an array of `s_mc_message_simcall_probe_one_t[n_transitions]
-                                          // is sent right after a `s_mc_message_actors_status_one_t`
+struct s_mc_message_simcall_probe_one_t { // a series of `s_mc_message_simcall_probe_one_t`
+                                          // is sent right after `s_mc_message_actors_status_one_t[]`
   std::array<char, SIMCALL_SERIALIZATION_BUFFER_SIZE> buffer;
 };
 
index ea0ea36..3e84903 100644 (file)
@@ -413,7 +413,8 @@ ExecPtr exec_async(double flops)
 
 aid_t get_pid()
 {
-  return simgrid::kernel::actor::ActorImpl::self()->get_pid();
+  const auto* self = simgrid::kernel::actor::ActorImpl::self();
+  return self ? self->get_pid() : 0;
 }
 
 aid_t get_ppid()
@@ -428,7 +429,8 @@ std::string get_name()
 
 const char* get_cname()
 {
-  return simgrid::kernel::actor::ActorImpl::self()->get_cname();
+  const auto* self = simgrid::kernel::actor::ActorImpl::self();
+  return self ? self->get_cname() : nullptr;
 }
 
 Host* get_host()
@@ -753,6 +755,8 @@ aid_t sg_actor_self_get_ppid()
 
 const char* sg_actor_self_get_name()
 {
+  if (simgrid::s4u::Actor::is_maestro())
+    return "maestro";
   return simgrid::s4u::this_actor::get_cname();
 }
 
index c20cd8e..a782de7 100644 (file)
@@ -42,20 +42,20 @@ void Engine::initialize(int* argc, char** argv)
   xbt_assert(Engine::instance_ == nullptr, "It is currently forbidden to create more than one instance of s4u::Engine");
   Engine::instance_ = this;
   instr::init();
-  pimpl->initialize(argc, argv);
+  pimpl_->initialize(argc, argv);
   // Either create a new context with maestro or create
   // a context object with the current context maestro):
   kernel::actor::create_maestro(maestro_code);
 }
 
-Engine::Engine(std::string name) : pimpl(new kernel::EngineImpl())
+Engine::Engine(std::string name) : pimpl_(new kernel::EngineImpl())
 {
   int argc   = 1;
   char* argv = &name[0];
   initialize(&argc, &argv);
 }
 
-Engine::Engine(int* argc, char** argv) : pimpl(new kernel::EngineImpl())
+Engine::Engine(int* argc, char** argv) : pimpl_(new kernel::EngineImpl())
 {
   initialize(argc, argv);
 }
@@ -81,6 +81,10 @@ Engine* Engine::get_instance(int* argc, char** argv)
   }
   return Engine::instance_;
 }
+const std::vector<std::string>& Engine::get_cmdline() const
+{
+  return pimpl_->get_cmdline();
+}
 
 void Engine::shutdown() // XBT_ATTRIB_DEPRECATED_v335
 {
@@ -99,22 +103,22 @@ double Engine::get_clock()
 void Engine::add_model(std::shared_ptr<kernel::resource::Model> model,
                        const std::vector<kernel::resource::Model*>& dependencies)
 {
-  kernel::actor::simcall_answered([this, &model, &dependencies] { pimpl->add_model(std::move(model), dependencies); });
+  kernel::actor::simcall_answered([this, &model, &dependencies] { pimpl_->add_model(std::move(model), dependencies); });
 }
 
 const std::vector<simgrid::kernel::resource::Model*>& Engine::get_all_models() const
 {
-  return pimpl->get_all_models();
+  return pimpl_->get_all_models();
 }
 
 void Engine::load_platform(const std::string& platf) const
 {
-  pimpl->load_platform(platf);
+  pimpl_->load_platform(platf);
 }
 
 void Engine::seal_platform() const
 {
-  pimpl->seal_platform();
+  pimpl_->seal_platform();
 }
 
 static void flatify_hosts(Engine const& engine, std::stringstream& ss)
@@ -283,12 +287,12 @@ void Engine::register_default(const std::function<void(int, char**)>& code)
 }
 void Engine::register_default(const kernel::actor::ActorCodeFactory& code)
 {
-  simgrid::kernel::actor::simcall_answered([this, &code]() { pimpl->register_default(code); });
+  simgrid::kernel::actor::simcall_answered([this, &code]() { pimpl_->register_default(code); });
 }
 
 void Engine::register_function(const std::string& name, const kernel::actor::ActorCodeFactory& code)
 {
-  simgrid::kernel::actor::simcall_answered([this, name, &code]() { pimpl->register_function(name, code); });
+  simgrid::kernel::actor::simcall_answered([this, name, &code]() { pimpl_->register_function(name, code); });
 }
 
 /** Load a deployment file and launch the actors that it contains
@@ -299,7 +303,7 @@ void Engine::register_function(const std::string& name, const kernel::actor::Act
  */
 void Engine::load_deployment(const std::string& deploy) const
 {
-  pimpl->load_deployment(deploy);
+  pimpl_->load_deployment(deploy);
 }
 
 /** Returns the amount of hosts in the platform */
@@ -316,8 +320,8 @@ std::vector<Host*> Engine::get_all_hosts() const
 std::vector<Host*> Engine::get_filtered_hosts(const std::function<bool(Host*)>& filter) const
 {
   std::vector<Host*> hosts;
-  if (pimpl->netzone_root_) {
-    hosts = pimpl->netzone_root_->get_filtered_hosts(filter);
+  if (pimpl_->netzone_root_) {
+    hosts = pimpl_->netzone_root_->get_filtered_hosts(filter);
   }
   /* Sort hosts in lexicographical order: keep same behavior when the hosts were saved on Engine
    * Some tests do a get_all_hosts() and selects hosts in this order */
@@ -342,8 +346,8 @@ Host* Engine::host_by_name(const std::string& name) const
 Host* Engine::host_by_name_or_null(const std::string& name) const
 {
   Host* host = nullptr;
-  if (pimpl->netzone_root_) {
-    auto* host_impl = pimpl->netzone_root_->get_host_by_name_or_null(name);
+  if (pimpl_->netzone_root_) {
+    auto* host_impl = pimpl_->netzone_root_->get_host_by_name_or_null(name);
     if (host_impl)
       host = host_impl->get_iface();
   }
@@ -364,7 +368,8 @@ Link* Engine::link_by_name(const std::string& name) const
 
 SplitDuplexLink* Engine::split_duplex_link_by_name(const std::string& name) const
 {
-  auto* link_impl = pimpl->netzone_root_ ? pimpl->netzone_root_->get_split_duplex_link_by_name_or_null(name) : nullptr;
+  auto* link_impl =
+      pimpl_->netzone_root_ ? pimpl_->netzone_root_->get_split_duplex_link_by_name_or_null(name) : nullptr;
   if (not link_impl)
     throw std::invalid_argument("Link not found: " + name);
   return link_impl->get_iface();
@@ -374,11 +379,11 @@ SplitDuplexLink* Engine::split_duplex_link_by_name(const std::string& name) cons
 Link* Engine::link_by_name_or_null(const std::string& name) const
 {
   Link* link = nullptr;
-  if (pimpl->netzone_root_) {
+  if (pimpl_->netzone_root_) {
     /* keep behavior where internal __loopback__ link from network model is given to user */
     if (name == "__loopback__")
-      return pimpl->netzone_root_->get_network_model()->loopback_->get_iface();
-    auto* link_impl = pimpl->netzone_root_->get_link_by_name_or_null(name);
+      return pimpl_->netzone_root_->get_network_model()->loopback_->get_iface();
+    auto* link_impl = pimpl_->netzone_root_->get_link_by_name_or_null(name);
     if (link_impl)
       link = link_impl->get_iface();
   }
@@ -390,7 +395,7 @@ Mailbox* Engine::mailbox_by_name_or_create(const std::string& name) const
 {
   /* two actors may have pushed the same mbox_create simcall at the same time */
   kernel::activity::MailboxImpl* mbox = kernel::actor::simcall_answered([&name, this] {
-    auto [m, inserted] = pimpl->mailboxes_.try_emplace(name, nullptr);
+    auto [m, inserted] = pimpl_->mailboxes_.try_emplace(name, nullptr);
     if (inserted) {
       m->second = new kernel::activity::MailboxImpl(name);
       XBT_DEBUG("Creating a mailbox at %p with name %s", m->second, name.c_str());
@@ -404,10 +409,10 @@ Mailbox* Engine::mailbox_by_name_or_create(const std::string& name) const
 size_t Engine::get_link_count() const
 {
   int count = 0;
-  if (pimpl->netzone_root_) {
-    count += pimpl->netzone_root_->get_link_count();
+  if (pimpl_->netzone_root_) {
+    count += pimpl_->netzone_root_->get_link_count();
     /* keep behavior where internal __loopback__ link from network model is given to user */
-    count += pimpl->netzone_root_->get_network_model()->loopback_ ? 1 : 0;
+    count += pimpl_->netzone_root_->get_network_model()->loopback_ ? 1 : 0;
   }
   return count;
 }
@@ -421,25 +426,25 @@ std::vector<Link*> Engine::get_all_links() const
 std::vector<Link*> Engine::get_filtered_links(const std::function<bool(Link*)>& filter) const
 {
   std::vector<Link*> res;
-  if (pimpl->netzone_root_) {
-    res = pimpl->netzone_root_->get_filtered_links(filter);
+  if (pimpl_->netzone_root_) {
+    res = pimpl_->netzone_root_->get_filtered_links(filter);
     /* keep behavior where internal __loopback__ link from network model is given to user */
-    if (pimpl->netzone_root_->get_network_model()->loopback_ &&
-        filter(pimpl->netzone_root_->get_network_model()->loopback_->get_iface()))
-      res.push_back(pimpl->netzone_root_->get_network_model()->loopback_->get_iface());
+    if (pimpl_->netzone_root_->get_network_model()->loopback_ &&
+        filter(pimpl_->netzone_root_->get_network_model()->loopback_->get_iface()))
+      res.push_back(pimpl_->netzone_root_->get_network_model()->loopback_->get_iface());
   }
   return res;
 }
 
 size_t Engine::get_actor_count() const
 {
-  return pimpl->get_actor_count();
+  return pimpl_->get_actor_count();
 }
 
 std::vector<ActorPtr> Engine::get_all_actors() const
 {
   std::vector<ActorPtr> actor_list;
-  for (auto const& [_, actor] : pimpl->get_actor_list()) {
+  for (auto const& [_, actor] : pimpl_->get_actor_list()) {
     actor_list.push_back(actor->get_iface());
   }
   return actor_list;
@@ -448,7 +453,7 @@ std::vector<ActorPtr> Engine::get_all_actors() const
 std::vector<ActorPtr> Engine::get_filtered_actors(const std::function<bool(ActorPtr)>& filter) const
 {
   std::vector<ActorPtr> actor_list;
-  for (auto const& [_, actor] : pimpl->get_actor_list()) {
+  for (auto const& [_, actor] : pimpl_->get_actor_list()) {
     if (filter(actor->get_iface()))
       actor_list.push_back(actor->get_iface());
   }
@@ -469,7 +474,7 @@ void Engine::run_until(double max_date) const
   fflush(stdout);
   fflush(stderr);
 
-  pimpl->run(max_date);
+  pimpl_->run(max_date);
 }
 
 void Engine::track_vetoed_activities(std::set<Activity*>* vetoed_activities) const
@@ -480,15 +485,15 @@ void Engine::track_vetoed_activities(std::set<Activity*>* vetoed_activities) con
 /** @brief Retrieve the root netzone, containing all others */
 s4u::NetZone* Engine::get_netzone_root() const
 {
-  if (pimpl->netzone_root_)
-    return pimpl->netzone_root_->get_iface();
+  if (pimpl_->netzone_root_)
+    return pimpl_->netzone_root_->get_iface();
   return nullptr;
 }
 /** @brief Set the root netzone, containing all others. Once set, it cannot be changed. */
 void Engine::set_netzone_root(const s4u::NetZone* netzone)
 {
-  xbt_assert(pimpl->netzone_root_ == nullptr, "The root NetZone cannot be changed once set");
-  pimpl->netzone_root_ = netzone->get_impl();
+  xbt_assert(pimpl_->netzone_root_ == nullptr, "The root NetZone cannot be changed once set");
+  pimpl_->netzone_root_ = netzone->get_impl();
 }
 
 static NetZone* netzone_by_name_recursive(NetZone* current, const std::string& name)
@@ -514,8 +519,8 @@ NetZone* Engine::netzone_by_name_or_null(const std::string& name) const
 /** @brief Retrieve the netpoint of the given name (or nullptr if not found) */
 kernel::routing::NetPoint* Engine::netpoint_by_name_or_null(const std::string& name) const
 {
-  auto netp = pimpl->netpoints_.find(name);
-  return netp == pimpl->netpoints_.end() ? nullptr : netp->second;
+  auto netp = pimpl_->netpoints_.find(name);
+  return netp == pimpl_->netpoints_.end() ? nullptr : netp->second;
 }
 
 kernel::routing::NetPoint* Engine::netpoint_by_name(const std::string& name) const
@@ -530,7 +535,7 @@ kernel::routing::NetPoint* Engine::netpoint_by_name(const std::string& name) con
 std::vector<kernel::routing::NetPoint*> Engine::get_all_netpoints() const
 {
   std::vector<kernel::routing::NetPoint*> res;
-  for (auto const& [_, netpoint] : pimpl->netpoints_)
+  for (auto const& [_, netpoint] : pimpl_->netpoints_)
     res.push_back(netpoint);
   return res;
 }
@@ -538,14 +543,14 @@ std::vector<kernel::routing::NetPoint*> Engine::get_all_netpoints() const
 /** @brief Register a new netpoint to the system */
 void Engine::netpoint_register(kernel::routing::NetPoint* point)
 {
-  simgrid::kernel::actor::simcall_answered([this, point] { pimpl->netpoints_[point->get_name()] = point; });
+  simgrid::kernel::actor::simcall_answered([this, point] { pimpl_->netpoints_[point->get_name()] = point; });
 }
 
 /** @brief Unregister a given netpoint */
 void Engine::netpoint_unregister(kernel::routing::NetPoint* point)
 {
   kernel::actor::simcall_answered([this, point] {
-    pimpl->netpoints_.erase(point->get_name());
+    pimpl_->netpoints_.erase(point->get_name());
     delete point;
   });
 }
index ddaec3f..fb1df9d 100644 (file)
@@ -137,6 +137,7 @@ void sg_config_init(int *argc, char **argv)
     XBT_WARN("Call to sg_config_init() after initialization ignored");
     return;
   }
+  _sg_cfg_init_status = 1;
 
   /* Plugins and models configuration */
   simgrid_plugins().create_flag("plugin", "The plugins", "", true);
@@ -234,8 +235,6 @@ void sg_config_init(int *argc, char **argv)
   static simgrid::config::Flag<bool> cfg_execution_cutpath{
       "exception/cutpath", "Whether to cut all path information from call traces, used e.g. in exceptions.", false};
 
-  _sg_cfg_init_status = 1;
-
   sg_config_cmd_line(argc, argv);
 
   xbt_mallocator_initialization_is_done(simgrid::kernel::context::Context::is_parallel());
index d2d121e..fdfcb18 100644 (file)
@@ -5,9 +5,9 @@
 
 #include "private.hpp"
 #include "src/internal_config.h"
+#include "src/kernel/EngineImpl.hpp"
 #include "src/smpi/include/smpi_actor.hpp"
 #include "src/xbt/memory_map.hpp"
-#include "xbt/virtu.h"
 
 #include <algorithm>
 #include <cerrno>
@@ -52,8 +52,9 @@ void smpi_prepare_global_memory_segment()
 
 static void smpi_get_executable_global_size()
 {
-  char* buffer = realpath(simgrid::xbt::binary_name.c_str(), nullptr);
-  xbt_assert(buffer != nullptr, "Could not resolve real path of binary file '%s'", simgrid::xbt::binary_name.c_str());
+  auto* binary_name = simgrid::kernel::EngineImpl::get_instance()->get_cmdline().front().c_str();
+  char* buffer      = realpath(binary_name, nullptr);
+  xbt_assert(buffer != nullptr, "Could not resolve real path of binary file '%s'", binary_name);
   std::string full_name = buffer;
   free(buffer);
 
index 57b6170..0a78eed 100644 (file)
@@ -17,6 +17,8 @@
 #include "simgrid/s4u/Host.hpp"
 #include "simgrid/plugins/file_system.h"
 
+#include <mutex> // std::scoped_lock
+
 #define FP_SIZE sizeof(MPI_Offset)
 
 XBT_LOG_NEW_DEFAULT_SUBCATEGORY(smpi_io, smpi, "Logging specific to SMPI (RMA operations)");
@@ -117,9 +119,8 @@ int File::get_position(MPI_Offset* offset) const
 
 int File::get_position_shared(MPI_Offset* offset) const
 {
-  shared_mutex_->lock();
+  const std::scoped_lock lock(*shared_mutex_);
   *offset = *shared_file_pointer_/etype_->get_extent();
-  shared_mutex_->unlock();
   return MPI_SUCCESS;
 }
 
@@ -146,10 +147,9 @@ int File::seek(MPI_Offset offset, int whence)
 
 int File::seek_shared(MPI_Offset offset, int whence)
 {
-  shared_mutex_->lock();
+  const std::scoped_lock lock(*shared_mutex_);
   seek(offset, whence);
   *shared_file_pointer_ = file_->tell();
-  shared_mutex_->unlock();
   return MPI_SUCCESS;
 }
 
@@ -184,11 +184,11 @@ int File::read(MPI_File fh, void* /*buf*/, int count, const Datatype* datatype,
 /* }*/
 int File::read_shared(MPI_File fh, void* buf, int count, const Datatype* datatype, MPI_Status* status)
 {
-  fh->shared_mutex_->lock();
-  fh->seek(*(fh->shared_file_pointer_), MPI_SEEK_SET);
-  read(fh, buf, count, datatype, status);
-  *(fh->shared_file_pointer_) = fh->file_->tell();
-  fh->shared_mutex_->unlock();
+  if (const std::scoped_lock lock(*fh->shared_mutex_); true) {
+    fh->seek(*(fh->shared_file_pointer_), MPI_SEEK_SET);
+    read(fh, buf, count, datatype, status);
+    *(fh->shared_file_pointer_) = fh->file_->tell();
+  }
   fh->seek(*(fh->shared_file_pointer_), MPI_SEEK_SET);
   return MPI_SUCCESS;
 }
@@ -210,9 +210,8 @@ int File::read_ordered(MPI_File fh, void* buf, int count, const Datatype* dataty
   fh->seek(result, MPI_SEEK_SET);
   int ret = fh->op_all<simgrid::smpi::File::read>(buf, count, datatype, status);
   if (fh->comm_->rank() == fh->comm_->size() - 1) {
-    fh->shared_mutex_->lock();
+    const std::scoped_lock lock(*fh->shared_mutex_);
     *(fh->shared_file_pointer_)=fh->file_->tell();
-    fh->shared_mutex_->unlock();
   }
   char c;
   simgrid::smpi::colls::bcast(&c, 1, MPI_BYTE, fh->comm_->size() - 1, fh->comm_);
@@ -241,14 +240,13 @@ int File::write(MPI_File fh, void* /*buf*/, int count, const Datatype* datatype,
 
 int File::write_shared(MPI_File fh, const void* buf, int count, const Datatype* datatype, MPI_Status* status)
 {
-  fh->shared_mutex_->lock();
+  const std::scoped_lock lock(*fh->shared_mutex_);
   XBT_DEBUG("Write shared on %s - Shared ptr before : %lld", fh->file_->get_path(), *(fh->shared_file_pointer_));
   fh->seek(*(fh->shared_file_pointer_), MPI_SEEK_SET);
   write(fh, const_cast<void*>(buf), count, datatype, status);
   *(fh->shared_file_pointer_) = fh->file_->tell();
   XBT_DEBUG("Write shared on %s - Shared ptr after : %lld", fh->file_->get_path(), *(fh->shared_file_pointer_));
   fh->seek(*(fh->shared_file_pointer_), MPI_SEEK_SET);
-  fh->shared_mutex_->unlock();
   return MPI_SUCCESS;
 }
 
@@ -268,9 +266,8 @@ int File::write_ordered(MPI_File fh, const void* buf, int count, const Datatype*
   fh->seek(result, MPI_SEEK_SET);
   int ret = fh->op_all<simgrid::smpi::File::write>(const_cast<void*>(buf), count, datatype, status);
   if (fh->comm_->rank() == fh->comm_->size() - 1) {
-    fh->shared_mutex_->lock();
+    const std::scoped_lock lock(*fh->shared_mutex_);
     *(fh->shared_file_pointer_)=fh->file_->tell();
-    fh->shared_mutex_->unlock();
   }
   char c;
   simgrid::smpi::colls::bcast(&c, 1, MPI_BYTE, fh->comm_->size() - 1, fh->comm_);
index 1a24c54..f2a700d 100644 (file)
@@ -24,6 +24,7 @@
 
 #include <algorithm>
 #include <array>
+#include <mutex> // std::scoped_lock and std::unique_lock
 
 XBT_LOG_NEW_DEFAULT_SUBCATEGORY(smpi_request, smpi, "Logging specific to SMPI (request)");
 
@@ -463,9 +464,9 @@ void Request::start()
 
     simgrid::smpi::ActorExt* process = smpi_process_remote(simgrid::s4u::Actor::by_pid(dst_));
 
-    simgrid::s4u::MutexPtr mut = process->mailboxes_mutex();
+    std::unique_lock<s4u::Mutex> mut_lock;
     if (smpi_cfg_async_small_thresh() != 0 || (flags_ & MPI_REQ_RMA) != 0)
-      mut->lock();
+      mut_lock = std::unique_lock(*process->mailboxes_mutex());
 
     bool is_probe = ((flags_ & MPI_REQ_PROBE) != 0);
     flags_ |= MPI_REQ_PROBE;
@@ -520,9 +521,6 @@ void Request::start()
                                               &observer);
 
     XBT_DEBUG("recv simcall posted");
-
-    if (smpi_cfg_async_small_thresh() != 0 || (flags_ & MPI_REQ_RMA) != 0)
-      mut->unlock();
   } else { /* the RECV flag was not set, so this is a send */
     const simgrid::smpi::ActorExt* process = smpi_process_remote(simgrid::s4u::Actor::by_pid(dst_));
     xbt_assert(process, "Actor pid=%ld is gone??", dst_);
@@ -574,10 +572,9 @@ void Request::start()
       XBT_DEBUG("sending size of %zu : sleep %f ", size_, sleeptime);
     }
 
-    simgrid::s4u::MutexPtr mut = process->mailboxes_mutex();
-
+    std::unique_lock<s4u::Mutex> mut_lock;
     if (smpi_cfg_async_small_thresh() != 0 || (flags_ & MPI_REQ_RMA) != 0)
-      mut->lock();
+      mut_lock = std::unique_lock(*process->mailboxes_mutex());
 
     if (not(smpi_cfg_async_small_thresh() != 0 || (flags_ & MPI_REQ_RMA) != 0)) {
       mailbox = process->mailbox();
@@ -629,9 +626,6 @@ void Request::start()
       boost::static_pointer_cast<kernel::activity::CommImpl>(action_)->set_tracing_category(
           smpi_process()->get_tracing_category());
     }
-
-    if (smpi_cfg_async_small_thresh() != 0 || ((flags_ & MPI_REQ_RMA) != 0))
-      mut->unlock();
   }
 }
 
@@ -1093,9 +1087,8 @@ int Request::wait(MPI_Request * request, MPI_Status * status)
 
   if ((*request)->flags_ & MPI_REQ_GENERALIZED) {
     if (not((*request)->flags_ & MPI_REQ_COMPLETE)) {
-      ((*request)->generalized_funcs)->mutex->lock();
-      ((*request)->generalized_funcs)->cond->wait(((*request)->generalized_funcs)->mutex);
-      ((*request)->generalized_funcs)->mutex->unlock();
+      const std::scoped_lock lock(*(*request)->generalized_funcs->mutex);
+      (*request)->generalized_funcs->cond->wait((*request)->generalized_funcs->mutex);
     }
     MPI_Status tmp_status;
     MPI_Status* mystatus;
@@ -1331,10 +1324,9 @@ int Request::grequest_complete(MPI_Request request)
 {
   if ((not(request->flags_ & MPI_REQ_GENERALIZED)) || request->generalized_funcs->mutex == nullptr)
     return MPI_ERR_REQUEST;
-  request->generalized_funcs->mutex->lock();
+  const std::scoped_lock lock(*request->generalized_funcs->mutex);
   request->flags_ |= MPI_REQ_COMPLETE; // in case wait would be called after complete
   request->generalized_funcs->cond->notify_one();
-  request->generalized_funcs->mutex->unlock();
   return MPI_SUCCESS;
 }
 
index c40ac06..7b33c80 100644 (file)
@@ -17,6 +17,7 @@
 #include "src/mc/mc_replay.hpp"
 
 #include <algorithm>
+#include <mutex> // std::scoped_lock
 
 XBT_LOG_NEW_DEFAULT_SUBCATEGORY(smpi_rma, smpi, "Logging specific to SMPI (RMA operations)");
 
@@ -239,16 +240,14 @@ int Win::put(const void *origin_addr, int origin_count, MPI_Datatype origin_data
     if(request!=nullptr){
       *request=sreq;
     }else{
-      mut_->lock();
+      const std::scoped_lock lock(*mut_);
       requests_.push_back(sreq);
-      mut_->unlock();
     }
 
     //push request to receiver's win
-    recv_win->mut_->lock();
+    const std::scoped_lock recv_lock(*recv_win->mut_);
     recv_win->requests_.push_back(rreq);
     rreq->start();
-    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);
@@ -283,9 +282,9 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
     //start the send, with another process than us as sender.
     sreq->start();
     // push request to sender's win
-    send_win->mut_->lock();
-    send_win->requests_.push_back(sreq);
-    send_win->mut_->unlock();
+    if (const std::scoped_lock send_lock(*send_win->mut_); true) {
+      send_win->requests_.push_back(sreq);
+    }
 
     //start recv
     rreq->start();
@@ -293,9 +292,8 @@ int Win::get( void *origin_addr, int origin_count, MPI_Datatype origin_datatype,
     if(request!=nullptr){
       *request=rreq;
     }else{
-      mut_->lock();
+      const std::scoped_lock lock(*mut_);
       requests_.push_back(rreq);
-      mut_->unlock();
     }
   } else {
     Datatype::copy(send_addr, target_count, target_datatype, origin_addr, origin_count, origin_datatype);
@@ -334,17 +332,16 @@ int Win::accumulate(const void *origin_addr, int origin_count, MPI_Datatype orig
   // 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 (const std::scoped_lock recv_lock(*recv_win->mut_); true) {
+    recv_win->requests_.push_back(rreq);
+    rreq->start();
+  }
 
   if (request != nullptr) {
     *request = sreq;
   } else {
-    mut_->lock();
+    const std::scoped_lock lock(*mut_);
     requests_.push_back(sreq);
-    mut_->unlock();
   }
 
   // FIXME: The current implementation fails to ensure the correct ordering of the accumulate requests.  The following
@@ -367,7 +364,7 @@ int Win::get_accumulate(const void* origin_addr, int origin_count, MPI_Datatype
   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_NULL;
-  send_win->atomic_mut_->lock();
+  const std::scoped_lock lock(*send_win->atomic_mut_);
   get(result_addr, result_count, result_datatype, target_rank,
               target_disp, target_count, target_datatype, &req);
   if (req != MPI_REQUEST_NULL)
@@ -377,7 +374,6 @@ int Win::get_accumulate(const void* origin_addr, int origin_count, MPI_Datatype
               target_disp, target_count, target_datatype, op, &req);
   if (req != MPI_REQUEST_NULL)
     Request::wait(&req, MPI_STATUS_IGNORE);
-  send_win->atomic_mut_->unlock();
   return MPI_SUCCESS;
 }
 
@@ -391,7 +387,7 @@ int Win::compare_and_swap(const void* origin_addr, const void* compare_addr, voi
 
   XBT_DEBUG("Entering MPI_Compare_and_swap with %d", target_rank);
   MPI_Request req = MPI_REQUEST_NULL;
-  send_win->atomic_mut_->lock();
+  const std::scoped_lock lock(*send_win->atomic_mut_);
   get(result_addr, 1, datatype, target_rank,
               target_disp, 1, datatype, &req);
   if (req != MPI_REQUEST_NULL)
@@ -400,7 +396,6 @@ int Win::compare_and_swap(const void* origin_addr, const void* compare_addr, voi
     put(origin_addr, 1, datatype, target_rank,
               target_disp, 1, datatype);
   }
-  send_win->atomic_mut_->unlock();
   return MPI_SUCCESS;
 }
 
@@ -614,7 +609,7 @@ int Win::finish_comms(){
   // 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();
+  const std::scoped_lock lock(*mut_);
   //Finish own requests
   int size = static_cast<int>(requests_.size());
   if (size > 0) {
@@ -622,13 +617,12 @@ int Win::finish_comms(){
     Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
     requests_.clear();
   }
-  mut_->unlock();
   return size;
 }
 
 int Win::finish_comms(int rank){
   // See comment about the mutex in finish_comms() above
-  mut_->lock();
+  const std::scoped_lock lock(*mut_);
   // 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.
@@ -646,7 +640,6 @@ int Win::finish_comms(int rank){
     Request::waitall(size, treqs, MPI_STATUSES_IGNORE);
     myreqqs.clear();
   }
-  mut_->unlock();
   return size;
 }
 
index 3552ec2..4d04e09 100644 (file)
@@ -1,3 +1,8 @@
+/* Copyright (c) 2002-2023. 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. */
+
 /* SimGrid's pthread interposer. Redefinition of the pthread symbols (see the comment in sthread.h) */
 
 #define _GNU_SOURCE
@@ -31,6 +36,9 @@ static sem_t* (*raw_sem_open)(const char*, int);
 static int (*raw_sem_init)(sem_t*, int, unsigned int);
 static int (*raw_sem_wait)(sem_t*);
 static int (*raw_sem_post)(sem_t*);
+static int (*raw_sem_destroy)(sem_t*);
+static int (*raw_sem_trywait)(sem_t*);
+static int (*raw_sem_timedwait)(sem_t*, const struct timespec*);
 
 static void intercepter_init()
 {
@@ -50,6 +58,9 @@ static void intercepter_init()
   raw_sem_init = dlsym(RTLD_NEXT, "sem_init");
   raw_sem_wait = dlsym(RTLD_NEXT, "sem_wait");
   raw_sem_post = dlsym(RTLD_NEXT, "sem_post");
+  raw_sem_destroy   = dlsym(RTLD_NEXT, "sem_destroy");
+  raw_sem_trywait   = dlsym(RTLD_NEXT, "sem_trywait");
+  raw_sem_timedwait = dlsym(RTLD_NEXT, "sem_timedwait");
 }
 
 static int sthread_inside_simgrid = 1;
@@ -155,6 +166,84 @@ int pthread_mutex_destroy(pthread_mutex_t* mutex)
   sthread_enable();
   return res;
 }
+int sem_init(sem_t* sem, int pshared, unsigned int value)
+{
+  if (raw_sem_init == NULL)
+    intercepter_init();
+
+  if (sthread_inside_simgrid)
+    return raw_sem_init(sem, pshared, value);
+
+  sthread_disable();
+  int res = sthread_sem_init((sthread_sem_t*)sem, pshared, value);
+  sthread_enable();
+  return res;
+}
+int sem_destroy(sem_t* sem)
+{
+  if (raw_sem_destroy == NULL)
+    intercepter_init();
+
+  if (sthread_inside_simgrid)
+    return raw_sem_destroy(sem);
+
+  sthread_disable();
+  int res = sthread_sem_destroy((sthread_sem_t*)sem);
+  sthread_enable();
+  return res;
+}
+int sem_post(sem_t* sem)
+{
+  if (raw_sem_post == NULL)
+    intercepter_init();
+
+  if (sthread_inside_simgrid)
+    return raw_sem_post(sem);
+
+  sthread_disable();
+  int res = sthread_sem_post((sthread_sem_t*)sem);
+  sthread_enable();
+  return res;
+}
+int sem_wait(sem_t* sem)
+{
+  if (raw_sem_wait == NULL)
+    intercepter_init();
+
+  if (sthread_inside_simgrid)
+    return raw_sem_wait(sem);
+
+  sthread_disable();
+  int res = sthread_sem_wait((sthread_sem_t*)sem);
+  sthread_enable();
+  return res;
+}
+int sem_trywait(sem_t* sem)
+{
+  if (raw_sem_trywait == NULL)
+    intercepter_init();
+
+  if (sthread_inside_simgrid)
+    return raw_sem_trywait(sem);
+
+  sthread_disable();
+  int res = sthread_sem_trywait((sthread_sem_t*)sem);
+  sthread_enable();
+  return res;
+}
+int sem_timedwait(sem_t* sem, const struct timespec* abs_timeout)
+{
+  if (raw_sem_timedwait == NULL)
+    intercepter_init();
+
+  if (sthread_inside_simgrid)
+    return raw_sem_timedwait(sem, abs_timeout);
+
+  sthread_disable();
+  int res = sthread_sem_timedwait((sthread_sem_t*)sem, abs_timeout);
+  sthread_enable();
+  return res;
+}
 
 /* Glibc < 2.31 uses type "struct timezone *" for the second parameter of gettimeofday.
    Other implementations use "void *" instead. */
@@ -210,24 +299,6 @@ int usleep(useconds_t usec)
 }
 
 #if 0
-int sem_init(sem_t *sem, int pshared, unsigned int value) {
-       int res;
-
-       res=raw_sem_init(sem,pshared,value);
-       return res;
-}
-
-int sem_wait(sem_t *sem) {
-       int res;
-
-       res = raw_sem_wait(sem);
-       return res;
-}
-
-int sem_post(sem_t *sem) {
-       return raw_sem_post(sem);
-}
-
 int pthread_cond_init(pthread_cond_t *cond, pthread_condattr_t *cond_attr) {
     *cond = sg_cond_init();
     return 0;
index 3bda35c..da9b453 100644 (file)
@@ -1,3 +1,8 @@
+/* Copyright (c) 2002-2023. 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. */
+
 /* SimGrid's pthread interposer. Intercepts most of the pthread and semaphore calls to model-check them.
  *
  * Intercepting on pthread is somewhat complicated by the fact that pthread is used everywhere in the system headers.
@@ -35,6 +40,16 @@ int sthread_mutex_trylock(sthread_mutex_t* mutex);
 int sthread_mutex_unlock(sthread_mutex_t* mutex);
 int sthread_mutex_destroy(sthread_mutex_t* mutex);
 
+typedef struct {
+  void* sem;
+} sthread_sem_t;
+int sthread_sem_init(sthread_sem_t* sem, int pshared, unsigned int value);
+int sthread_sem_destroy(sthread_sem_t* sem);
+int sthread_sem_post(sthread_sem_t* sem);
+int sthread_sem_wait(sthread_sem_t* sem);
+int sthread_sem_trywait(sthread_sem_t* sem);
+int sthread_sem_timedwait(sthread_sem_t* sem, const struct timespec* abs_timeout);
+
 int sthread_gettimeofday(struct timeval* tv);
 void sthread_sleep(double seconds);
 
index c4aa27d..8076231 100644 (file)
@@ -1,3 +1,8 @@
+/* Copyright (c) 2002-2023. 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. */
+
 /* SimGrid's pthread interposer. Actual implementation of the symbols (see the comment in sthread.h) */
 
 #include "smpi/smpi.h"
@@ -7,6 +12,7 @@
 #include <simgrid/s4u/Engine.hpp>
 #include <simgrid/s4u/Mutex.hpp>
 #include <simgrid/s4u/NetZone.hpp>
+#include <simgrid/s4u/Semaphore.hpp>
 #include <xbt/base.h>
 #include <xbt/sysdep.h>
 
@@ -146,6 +152,48 @@ int sthread_mutex_destroy(sthread_mutex_t* mutex)
   intrusive_ptr_release(static_cast<sg4::Mutex*>(mutex->mutex));
   return 0;
 }
+int sthread_sem_init(sthread_sem_t* sem, int /*pshared*/, unsigned int value)
+{
+  auto s = sg4::Semaphore::create(value);
+  intrusive_ptr_add_ref(s.get());
+
+  sem->sem = s.get();
+  return 0;
+}
+int sthread_sem_destroy(sthread_sem_t* sem)
+{
+  intrusive_ptr_release(static_cast<sg4::Semaphore*>(sem->sem));
+  return 0;
+}
+int sthread_sem_post(sthread_sem_t* sem)
+{
+  static_cast<sg4::Semaphore*>(sem->sem)->release();
+  return 0;
+}
+int sthread_sem_wait(sthread_sem_t* sem)
+{
+  static_cast<sg4::Semaphore*>(sem->sem)->acquire();
+  return 0;
+}
+int sthread_sem_trywait(sthread_sem_t* sem)
+{
+  auto* s = static_cast<sg4::Semaphore*>(sem->sem);
+  if (s->would_block()) {
+    errno = EAGAIN;
+    return -1;
+  }
+  s->acquire();
+  return 0;
+}
+int sthread_sem_timedwait(sthread_sem_t* sem, const struct timespec* abs_timeout)
+{
+  if (static_cast<sg4::Semaphore*>(sem->sem)->acquire_timeout(static_cast<double>(abs_timeout->tv_sec) +
+                                                              static_cast<double>(abs_timeout->tv_nsec) / 1E9)) {
+    errno = ETIMEDOUT;
+    return -1;
+  }
+  return 0;
+}
 
 int sthread_gettimeofday(struct timeval* tv)
 {
@@ -165,24 +213,6 @@ void sthread_sleep(double seconds)
 }
 
 #if 0
-int sem_init(sem_t *sem, int pshared, unsigned int value) {
-       int res;
-
-       res=raw_sem_init(sem,pshared,value);
-       return res;
-}
-
-int sem_wait(sem_t *sem) {
-       int res;
-
-       res = raw_sem_wait(sem);
-       return res;
-}
-
-int sem_post(sem_t *sem) {
-       return raw_sem_post(sem);
-}
-
 int pthread_cond_init(pthread_cond_t *cond, pthread_condattr_t *cond_attr) {
     *cond = sg_cond_init();
     return 0;
index 2b7e1f2..821075f 100644 (file)
@@ -22,7 +22,7 @@ public:
 
   inline void release()
   {
-    std::unique_lock lock(mutex_);
+    const std::scoped_lock lock(mutex_);
     ++capa_;
     condition_.notify_one();
   }
index 24ebc39..4914d2b 100644 (file)
@@ -5,10 +5,11 @@
 
 #include "src/internal_config.h"
 
+#include <simgrid/actor.h>
+#include <simgrid/s4u/Actor.hpp>
 #include <xbt/backtrace.hpp>
 #include <xbt/string.hpp>
 #include <xbt/sysdep.h>
-#include <xbt/virtu.h>
 
 #include <cstdio>
 #include <cstdlib>
@@ -90,7 +91,8 @@ std::string Backtrace::resolve() const
 void Backtrace::display() const
 {
   std::string backtrace = resolve();
-  std::fprintf(stderr, "Backtrace (displayed in actor %s%s):\n%s\n", xbt_procname(),
+  std::fprintf(stderr, "Backtrace (displayed in actor %s%s):\n%s\n",
+               simgrid::s4u::Actor::is_maestro() ? "maestro" : sg_actor_self_get_name(),
                (xbt_log_no_loc ? " -- short trace because of --log=no_loc" : ""),
                backtrace.empty() ? "(backtrace not set -- did you install Boost.Stacktrace?)" : backtrace.c_str());
 }
index a155e19..f1e73d3 100644 (file)
@@ -8,7 +8,6 @@
 #include "xbt/dict.h"
 #include "dict_private.h"
 #include "simgrid/Exception.hpp"
-#include "src/xbt/xbt_modinter.h"
 #include "xbt/ex.h"
 #include "xbt/log.h"
 #include "xbt/mallocator.h"
 #include <algorithm>
 #include <cstdio>
 #include <cstring>
+#include <mutex>
 
 XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_dict, xbt, "Dictionaries provide the same functionalities as hash tables");
 
 constexpr int MAX_FILL_PERCENT = 80;
 
+/** Handle the dict mallocators init/fini cycle. */
+static void xbt_dict_postexit()
+{
+  if (dict_elm_mallocator != nullptr) {
+    xbt_mallocator_free(dict_elm_mallocator);
+    dict_elm_mallocator = nullptr;
+  }
+}
+static void xbt_dict_preinit()
+{
+  static std::mutex init_mutex;
+  const std::scoped_lock lock(init_mutex);
+  if (dict_elm_mallocator == nullptr) {
+    dict_elm_mallocator =
+        xbt_mallocator_new(256, dict_elm_mallocator_new_f, dict_elm_mallocator_free_f, dict_elm_mallocator_reset_f);
+    atexit(xbt_dict_postexit);
+  }
+}
+
 /**
  * @brief Constructor
  * @param free_ctn function to call with (@a data as argument) when @a data is removed from the dictionary
@@ -34,7 +53,7 @@ constexpr int MAX_FILL_PERCENT = 80;
  */
 xbt_dict_t xbt_dict_new_homogeneous(void_f_pvoid_t free_ctn)
 {
-  xbt_dict_preinit();
+  xbt_dict_preinit(); // Make sure that the module is intialized
 
   xbt_dict_t dict;
 
@@ -304,27 +323,3 @@ int xbt_dict_is_empty(const_xbt_dict_t dict)
 {
   return not dict || (xbt_dict_length(dict) == 0);
 }
-
-/**
- * Create the dict mallocators.
- * This is an internal XBT function called during the lib initialization.
- * It can be used several times to recreate the mallocator, for example when you switch to MC mode
- */
-void xbt_dict_preinit()
-{
-  if (dict_elm_mallocator == nullptr)
-    dict_elm_mallocator = xbt_mallocator_new(256, dict_elm_mallocator_new_f, dict_elm_mallocator_free_f,
-      dict_elm_mallocator_reset_f);
-}
-
-/**
- * Destroy the dict mallocators.
- * This is an internal XBT function during the lib initialization
- */
-void xbt_dict_postexit()
-{
-  if (dict_elm_mallocator != nullptr) {
-    xbt_mallocator_free(dict_elm_mallocator);
-    dict_elm_mallocator = nullptr;
-  }
-}
index 5aa9eae..e8048f8 100644 (file)
@@ -17,9 +17,9 @@ XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_exception, xbt, "Exceptions");
 
 void _xbt_throw(char* message, const char* file, int line, const char* func)
 {
-  simgrid::Exception e(
-      simgrid::xbt::ThrowPoint(file, line, func, simgrid::xbt::Backtrace(), xbt_procname(), xbt_getpid()),
-      message ? message : "");
+  simgrid::Exception e(simgrid::xbt::ThrowPoint(file, line, func, simgrid::xbt::Backtrace(), sg_actor_self_get_name(),
+                                                sg_actor_self_get_pid()),
+                       message ? message : "");
   xbt_free(message);
   throw e;
 }
@@ -37,7 +37,7 @@ void log_exception(e_xbt_log_priority_t prio, const char* context, std::exceptio
 
     auto* with_context = dynamic_cast<const simgrid::Exception*>(&exception);
     if (with_context != nullptr) {
-      XBT_LOG(prio, "%s %s by %s/%d: %s", context, name.c_str(), with_context->throw_point().procname_.c_str(),
+      XBT_LOG(prio, "%s %s by %s/%ld: %s", context, name.c_str(), with_context->throw_point().procname_.c_str(),
               with_context->throw_point().pid_, exception.what());
       // Do we have a backtrace?
       if (not simgrid::config::get_value<bool>("exception/cutpath")) {
index e9c6542..d21ea0e 100644 (file)
@@ -6,7 +6,6 @@
  * under the terms of the license (GNU LGPL) which comes with this package. */
 
 #include "src/xbt/log_private.hpp"
-#include "src/xbt/xbt_modinter.h"
 #include "xbt/string.hpp"
 #include "xbt/sysdep.h"
 
@@ -56,6 +55,7 @@ XBT_LOG_NEW_DEFAULT_SUBCATEGORY(log, xbt, "Loggings from the logging mechanism i
 
 static void xbt_log_help();
 static void xbt_log_help_categories();
+static void xbt_log_postexit();
 
 void xbt_log_init(int *argc, char **argv)
 {
@@ -64,6 +64,7 @@ void xbt_log_init(int *argc, char **argv)
   int parse_args          = 1; // Stop parsing the parameters once we found '--'
 
   xbt_log_control_set("xbt_help.app:stdout xbt_help.threshold:VERBOSE xbt_help.fmt:%m%n");
+  atexit(xbt_log_postexit);
 
   /* Set logs and init log submodule */
   for (int i = 1; i < *argc; i++) {
@@ -97,24 +98,27 @@ void xbt_log_init(int *argc, char **argv)
   }
 }
 
-static void log_cat_exit(const s_xbt_log_category_t* cat)
+static void log_cat_exit(xbt_log_category_t cat)
 {
   if (cat->appender) {
     if (cat->appender->free_)
       cat->appender->free_(cat->appender);
     xbt_free(cat->appender);
+    cat->appender = nullptr;
   }
   if (cat->layout) {
     if (cat->layout->free_)
       cat->layout->free_(cat->layout);
     xbt_free(cat->layout);
+    cat->layout = nullptr;
   }
 
-  for (auto const* child = cat->firstChild; child != nullptr; child = child->nextSibling)
+  for (auto* child = cat->firstChild; child != nullptr; child = child->nextSibling)
     log_cat_exit(child);
+  cat->firstChild = nullptr;
 }
 
-void xbt_log_postexit(void)
+static void xbt_log_postexit(void)
 {
   XBT_VERB("Exiting log");
   log_cat_exit(&_XBT_LOGV(XBT_LOG_ROOT_CAT));
@@ -231,7 +235,7 @@ int _xbt_log_cat_init(xbt_log_category_t category, e_xbt_log_priority_t priority
     return priority >= category->threshold;
 
   static std::recursive_mutex log_cat_init_mutex;
-  log_cat_init_mutex.lock();
+  const std::scoped_lock lock(log_cat_init_mutex);
 
   XBT_DEBUG("Initializing category '%s' (firstChild=%s, nextSibling=%s)", category->name,
          (category->firstChild ? category->firstChild->name : "none"),
@@ -275,7 +279,6 @@ int _xbt_log_cat_init(xbt_log_category_t category, e_xbt_log_priority_t priority
   }
 
   category->initialized = 1;
-  log_cat_init_mutex.unlock();
   return priority >= category->threshold;
 }
 
index d8b5eb7..7c02fa1 100644 (file)
@@ -309,7 +309,7 @@ template <typename T> void Parmap<T>::worker_main(ThreadData* data)
 
 template <typename T> void Parmap<T>::PosixSynchro::master_signal()
 {
-  std::unique_lock lk(ready_mutex);
+  const std::scoped_lock lock(ready_mutex);
   this->parmap.thread_counter = 1;
   this->parmap.work_round++;
   /* wake all workers */
@@ -318,14 +318,14 @@ template <typename T> void Parmap<T>::PosixSynchro::master_signal()
 
 template <typename T> void Parmap<T>::PosixSynchro::master_wait()
 {
-  std::unique_lock lk(done_mutex);
+  std::unique_lock lock(done_mutex);
   /* wait for all workers to be ready */
-  done_cond.wait(lk, [this]() { return this->parmap.thread_counter >= this->parmap.num_workers; });
+  done_cond.wait(lock, [this]() { return this->parmap.thread_counter >= this->parmap.num_workers; });
 }
 
 template <typename T> void Parmap<T>::PosixSynchro::worker_signal()
 {
-  std::unique_lock lk(done_mutex);
+  const std::scoped_lock lock(done_mutex);
   this->parmap.thread_counter++;
   if (this->parmap.thread_counter == this->parmap.num_workers) {
     /* all workers have finished, wake the controller */
@@ -335,9 +335,9 @@ template <typename T> void Parmap<T>::PosixSynchro::worker_signal()
 
 template <typename T> void Parmap<T>::PosixSynchro::worker_wait(unsigned expected_round)
 {
-  std::unique_lock lk(ready_mutex);
+  std::unique_lock lock(ready_mutex);
   /* wait for more work */
-  ready_cond.wait(lk, [this, expected_round]() { return this->parmap.work_round == expected_round; });
+  ready_cond.wait(lock, [this, expected_round]() { return this->parmap.work_round == expected_round; });
 }
 
 #if HAVE_FUTEX_H
index 61bb830..e09d701 100644 (file)
@@ -9,7 +9,9 @@
 #include "simgrid/host.h"
 #include "src/xbt/log_private.hpp"
 #include "xbt/sysdep.h"
-#include "xbt/virtu.h"
+#include <simgrid/actor.h>
+#include <simgrid/s4u/Actor.hpp>
+
 #include <algorithm>
 #include <cstdio>
 
@@ -67,6 +69,7 @@ static constexpr const char* ERRMSG =
   } else                                                                                                               \
     (void)0
 #define show_int(data) show_it((data), "d")
+#define show_long(data) show_it((data), "ld")
 #define show_double(data) show_it((data), "f")
 
 static bool xbt_log_layout_format_doit(const s_xbt_log_layout_t* l, xbt_log_event_t ev, const char* msg_fmt)
@@ -134,10 +137,10 @@ static bool xbt_log_layout_format_doit(const s_xbt_log_layout_t* l, xbt_log_even
         case 't': /* thread/process name; LOG4J compliant */
         case 'P': /* Used before SimGrid 3.26 and kept for compatiblity. Should not hurt. */
         case 'a': /* actor name; SimGrid extension */
-          show_string(xbt_procname());
+          show_string(sg_actor_self_get_name());
           break;
         case 'i': /* actor ID; SimGrid extension */
-          show_int(xbt_getpid());
+          show_long(sg_actor_self_get_pid());
           break;
         case 'F': /* file name; LOG4J compliant */
           show_string(ev->fileName);
index a689123..da43e17 100644 (file)
@@ -7,7 +7,8 @@
 
 #include "src/xbt/log_private.hpp"
 #include "xbt/sysdep.h"
-#include "xbt/virtu.h"
+#include <simgrid/actor.h>
+#include <simgrid/s4u/Actor.hpp>
 
 #include "simgrid/engine.h" /* simgrid_get_clock */
 #include "simgrid/host.h"   /* sg_host_self_get_name */
@@ -34,12 +35,12 @@ static bool xbt_log_layout_simple_doit(const s_xbt_log_layout_t*, xbt_log_event_
   check_overflow(1);
 
   /* Display the proc info if available */
-  procname = xbt_procname();
+  procname = sg_actor_self_get_name();
   if (procname && strcmp(procname,"maestro")) {
-    len = snprintf(p, rem_size, "%s:%s:(%d) ", sg_host_self_get_name(), procname, xbt_getpid());
+    len = snprintf(p, rem_size, "%s:%s:(%ld) ", sg_host_self_get_name(), procname, sg_actor_self_get_pid());
     check_overflow(len);
   } else if (not procname) {
-    len = snprintf(p, rem_size, "%s::(%d) ", sg_host_self_get_name(), xbt_getpid());
+    len = snprintf(p, rem_size, "%s::(%ld) ", sg_host_self_get_name(), sg_actor_self_get_pid());
     check_overflow(len);
   }
 
diff --git a/src/xbt/xbt_main.cpp b/src/xbt/xbt_main.cpp
deleted file mode 100644 (file)
index ee40cfc..0000000
+++ /dev/null
@@ -1,125 +0,0 @@
-/* module handling                                                          */
-
-/* Copyright (c) 2006-2023. 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. */
-
-#define XBT_LOG_LOCALLY_DEFINE_XBT_CHANNEL /* MSVC don't want it to be declared extern in headers and local here */
-
-#include "simgrid/config.h"
-#include "src/internal_config.h"
-#include "src/simgrid/sg_config.hpp"
-#include "src/sthread/sthread.h" // sthread_inside_simgrid
-#include "src/xbt/coverage.h"
-#include "src/xbt/xbt_modinter.h" /* prototype of other module's init/exit in XBT */
-#include "xbt/config.hpp"
-#include "xbt/dynar.h"
-#include "xbt/log.h"
-#include "xbt/log.hpp"
-#include "xbt/misc.h"
-#include "xbt/module.h" /* this module */
-#include "xbt/sysdep.h"
-
-#include <cmath>
-#include <cstdio>
-#if HAVE_UNISTD_H
-# include <unistd.h>
-#endif
-#include <string>
-#include <vector>
-
-XBT_LOG_NEW_DEFAULT_SUBCATEGORY(module, xbt, "module handling");
-
-XBT_LOG_NEW_CATEGORY(smpi, "All SMPI categories"); /* lives here even if that's a bit odd to solve linking issues: this is used in xbt_log_file_appender to detect whether SMPI is used (and thus whether we should unbench the writing to disk) */
-
-namespace simgrid::xbt {
-std::string binary_name;          /* Name of the system process containing us (mandatory to retrieve neat backtraces) */
-std::vector<std::string> cmdline; /* all we got in argv */
-} // namespace simgrid::xbt
-
-
-int xbt_initialized = 0;
-simgrid::config::Flag<bool> cfg_dbg_clean_atexit{
-    "debug/clean-atexit",
-    "Whether to cleanup SimGrid at exit. Disable it if your code segfaults after its end.",
-    true};
-
-const int xbt_pagesize = static_cast<int>(sysconf(_SC_PAGESIZE));
-const int xbt_pagebits = static_cast<int>(log2(xbt_pagesize));
-
-/* Declare xbt_preinit and xbt_postexit as constructor/destructor of the library.
- * This is crude and rather compiler-specific, unfortunately.
- */
-static void xbt_preinit() XBT_ATTRIB_CONSTRUCTOR(200);
-static void xbt_postexit();
-XBT_ATTRIB_NOINLINE void sthread_enable()
-{ // These symbols are used from ContextSwapped in any case, but they are only useful
-  asm("");
-}
-XBT_ATTRIB_NOINLINE void sthread_disable()
-{ //  when libsthread is LD_PRELOADED. In this case, sthread's implem gets used instead.
-  asm("");
-}
-
-static void xbt_preinit()
-{
-  xbt_dict_preinit();
-  atexit(xbt_postexit);
-}
-
-static void xbt_postexit()
-{
-  if (not cfg_dbg_clean_atexit)
-    return;
-  xbt_initialized--;
-  xbt_dict_postexit();
-  xbt_log_postexit();
-}
-
-/** @brief Initialize the xbt mechanisms. */
-void xbt_init(int *argc, char **argv)
-{
-  xbt_initialized++;
-  if (xbt_initialized > 1) {
-    XBT_DEBUG("XBT has been initialized %d times.", xbt_initialized);
-    return;
-  }
-
-  simgrid::xbt::install_exception_handler();
-
-  if (*argc > 0)
-    simgrid::xbt::binary_name = argv[0];
-  for (int i = 0; i < *argc; i++)
-    simgrid::xbt::cmdline.emplace_back(argv[i]);
-
-  xbt_log_init(argc, argv);
-}
-
-/* these two functions belong to xbt/sysdep.h, which have no corresponding .c file */
-/** @brief like xbt_free, but you can be sure that it is a function  */
-void xbt_free_f(void* p) noexcept(noexcept(::free))
-{
-  xbt_free(p);
-}
-
-/** @brief should be given a pointer to pointer, and frees the second one */
-void xbt_free_ref(void* d) noexcept(noexcept(::free))
-{
-  xbt_free(*(void**)d);
-}
-
-/** @brief Kill the program in silence */
-void xbt_abort()
-{
-  /* Call __gcov_flush on abort when compiling with coverage options. */
-  coverage_checkpoint();
-  abort();
-}
-
-#ifndef HAVE_SMPI
-int SMPI_is_inited()
-{
-  return false;
-}
-#endif
diff --git a/src/xbt/xbt_misc.cpp b/src/xbt/xbt_misc.cpp
new file mode 100644 (file)
index 0000000..d5522f8
--- /dev/null
@@ -0,0 +1,64 @@
+/* Various pieces of code which don't fit in any module                     */
+
+/* Copyright (c) 2006-2023. 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. */
+
+#define XBT_LOG_LOCALLY_DEFINE_XBT_CHANNEL /* MSVC don't want it to be declared extern in headers and local here */
+
+#include "src/internal_config.h"
+#include "src/sthread/sthread.h" // sthread_inside_simgrid
+#include "src/xbt/coverage.h"
+#include "xbt/log.h"
+#include "xbt/misc.h"
+#include "xbt/sysdep.h"
+
+#include <cmath>
+#if HAVE_UNISTD_H
+#include <unistd.h>
+#endif
+
+XBT_LOG_NEW_CATEGORY(smpi, "All SMPI categories"); /* lives here even if that's a bit odd to solve linking issues: this
+                                                      is used in xbt_log_file_appender to detect whether SMPI is used
+                                                      (and thus whether we should unbench the writing to disk) */
+
+const int xbt_pagesize = static_cast<int>(sysconf(_SC_PAGESIZE));
+const int xbt_pagebits = static_cast<int>(log2(xbt_pagesize));
+
+XBT_ATTRIB_NOINLINE void sthread_enable()
+{ // These symbols are used from ContextSwapped in any case, but they are only useful
+  asm("");
+}
+XBT_ATTRIB_NOINLINE void sthread_disable()
+{ //  when libsthread is LD_PRELOADED. In this case, sthread's implem gets used instead.
+  asm("");
+}
+
+/* these two functions belong to xbt/sysdep.h, which have no corresponding .c file */
+/** @brief like xbt_free, but you can be sure that it is a function  */
+void xbt_free_f(void* p) noexcept(noexcept(::free))
+{
+  xbt_free(p);
+}
+
+/** @brief should be given a pointer to pointer, and frees the second one */
+void xbt_free_ref(void* d) noexcept(noexcept(::free))
+{
+  xbt_free(*(void**)d);
+}
+
+/** @brief Kill the program in silence */
+void xbt_abort()
+{
+  /* Call __gcov_flush on abort when compiling with coverage options. */
+  coverage_checkpoint();
+  abort();
+}
+
+#ifndef HAVE_SMPI
+int SMPI_is_inited()
+{
+  return false;
+}
+#endif
diff --git a/src/xbt/xbt_modinter.h b/src/xbt/xbt_modinter.h
deleted file mode 100644 (file)
index 6eee62f..0000000
+++ /dev/null
@@ -1,25 +0,0 @@
-/* xbt_modinter - How to init/exit the XBT modules                          */
-
-/* Copyright (c) 2004-2023. The SimGrid Team. All rights reserved.          */
-
-/* This program is free software; you can redistribute it and/or modify it
- * under the terms of the license (GNU LGPL) which comes with this package. */
-
-#ifndef XBT_MODINTER_H
-#define XBT_MODINTER_H
-#include "xbt/misc.h"
-
-SG_BEGIN_DECL
-
-/* Modules definitions */
-
-void xbt_log_postexit(void);
-
-void xbt_dict_preinit(void);
-void xbt_dict_postexit(void);
-
-extern int xbt_initialized;
-
-SG_END_DECL
-
-#endif
diff --git a/src/xbt/xbt_virtu.cpp b/src/xbt/xbt_virtu.cpp
deleted file mode 100644 (file)
index 56b2616..0000000
+++ /dev/null
@@ -1,22 +0,0 @@
-/* virtualization layer for XBT */
-
-/* Copyright (c) 2007-2023. 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 <simgrid/s4u/Actor.hpp>
-#include <xbt/virtu.h>
-
-#include "src/kernel/actor/ActorImpl.hpp"
-
-int xbt_getpid()
-{
-  const auto* self = simgrid::kernel::actor::ActorImpl::self();
-  return self == nullptr ? 0 : static_cast<int>(self->get_pid());
-}
-
-const char* xbt_procname(void)
-{
-  return simgrid::s4u::Actor::is_maestro() ? "maestro" : simgrid::kernel::actor::ActorImpl::self()->get_cname();
-}
index 22224b6..adb9519 100644 (file)
@@ -24,6 +24,8 @@
 #include "simgrid/s4u/Mailbox.hpp"
 #include "simgrid/s4u/Mutex.hpp"
 
+#include <mutex> // std::unique_lock
+
 XBT_LOG_NEW_DEFAULT_CATEGORY(mutex_handling, "Messages specific for this test");
 
 static int receiver(const char* box_name)
@@ -45,14 +47,11 @@ static int sender(const char* box_name, simgrid::s4u::MutexPtr mutex, int value)
   auto* payload = new int(value);
   auto mb      = simgrid::s4u::Mailbox::by_name(box_name);
 
+  std::unique_lock<simgrid::s4u::Mutex> lock;
   if (mutex)
-    mutex->lock();
+    lock = std::unique_lock(*mutex);
 
   mb->put(payload, 8);
-
-  if (mutex)
-    mutex->unlock();
-
   return 0;
 }
 
index c37cfdc..190d995 100644 (file)
@@ -8,7 +8,6 @@
 #include "simgrid/s4u/Engine.hpp"
 #include "src/kernel/lmm/maxmin.hpp"
 #include "xbt/log.h"
-#include "xbt/module.h"
 #include "xbt/sysdep.h"
 #include <algorithm>
 #include <array>
index d905956..b43d234 100644 (file)
@@ -7,7 +7,6 @@
 
 #include "simgrid/s4u/Engine.hpp"
 #include "src/kernel/lmm/maxmin.hpp"
-#include "xbt/module.h"
 #include "xbt/random.hpp"
 #include "xbt/sysdep.h" /* time manipulation for benchmarking */
 #include "xbt/xbt_os_time.h"
index 70317ad..83969f8 100644 (file)
@@ -17,7 +17,7 @@ foreach(x actor actor-autorestart actor-suspend
         storage_client_server listen_async pid
         trace-integration
         seal-platform
-             vm-live-migration vm-suicide issue71)
+        vm-live-migration vm-suicide issue71)
 
   if(NOT DEFINED ${x}_sources)
       set(${x}_sources ${x}/${x}.cpp)
index 3287b2e..6db4f0c 100644 (file)
@@ -63,7 +63,7 @@ def worker(my_id):
     assert Engine.clock < deadline, f"Failed to run all tasks in less than {deadline} seconds. Is this an infinite loop?"
 
     try:
-      this_actor.info(f"Waiting a message on mailbox")
+      this_actor.info("Waiting a message on mailbox")
       compute_cost = mailbox.get()
 
       this_actor.info("Start execution...")
index 3b03ea9..c76ea46 100644 (file)
@@ -4,6 +4,7 @@
  * under the terms of the license (GNU LGPL) which comes with this package. */
 
 #include "simgrid/Exception.hpp"
+#include "simgrid/engine.h"
 #include "src/xbt/mmalloc/mmalloc.h"
 #include "xbt.h"
 
@@ -34,7 +35,7 @@ int main(int argc, char**argv)
 {
   xbt_mheap_t heapA = nullptr;
   std::array<void*, TESTSIZE> pointers;
-  xbt_init(&argc,argv);
+  simgrid_init(&argc, argv);
 
   XBT_INFO("Allocating a new heap");
   unsigned long mask = ~((unsigned long)xbt_pagesize - 1);
index d13225e..0bf6946 100644 (file)
@@ -56,7 +56,6 @@ set(EXTRA_DIST
   src/xbt/log_private.hpp
   src/xbt/mallocator_private.h
   src/xbt/parmap.hpp
-  src/xbt/xbt_modinter.h
   
   src/xbt/mmalloc/mmalloc.h
   src/xbt/mmalloc/mfree.c
@@ -278,13 +277,12 @@ set(XBT_SRC
   src/xbt/xbt_log_appender_file.cpp
   src/xbt/xbt_log_layout_format.cpp
   src/xbt/xbt_log_layout_simple.cpp
-  src/xbt/xbt_main.cpp
+  src/xbt/xbt_misc.cpp
   src/xbt/xbt_os_file.cpp
   src/xbt/xbt_os_time.c
   src/xbt/xbt_parse_units.cpp
   src/xbt/xbt_replay.cpp
   src/xbt/xbt_str.cpp
-  src/xbt/xbt_virtu.cpp
   )
 
 if(HAVE_MMALLOC)