diff --git a/src/coordinator/coordinator_worker.hpp b/src/coordinator/coordinator_worker.hpp
index 46bc546cd..f77d9a263 100644
--- a/src/coordinator/coordinator_worker.hpp
+++ b/src/coordinator/coordinator_worker.hpp
@@ -71,6 +71,9 @@ struct QueueInner {
   // starvation by sometimes randomizing priorities, rather than following a strict
   // prioritization.
   std::deque<Message> queue;
+
+  uint64_t submitted = 0;
+  uint64_t calls_to_pop = 0;
 };
 
 /// There are two reasons to implement our own Queue instead of using
@@ -86,6 +89,8 @@ class Queue {
       MG_ASSERT(inner_.use_count() > 0);
       std::unique_lock<std::mutex> lock(inner_->mu);
 
+      inner_->submitted++;
+
       inner_->queue.emplace_back(std::move(message));
     }  // lock dropped before notifying condition variable
 
@@ -96,6 +101,9 @@ class Queue {
     MG_ASSERT(inner_.use_count() > 0);
     std::unique_lock<std::mutex> lock(inner_->mu);
 
+    inner_->calls_to_pop++;
+    inner_->cv.notify_all();
+
     while (inner_->queue.empty()) {
       inner_->cv.wait(lock);
     }
@@ -105,6 +113,15 @@ class Queue {
 
     return message;
   }
+
+  void BlockOnQuiescence() const {
+    MG_ASSERT(inner_.use_count() > 0);
+    std::unique_lock<std::mutex> lock(inner_->mu);
+
+    while (inner_->calls_to_pop <= inner_->submitted) {
+      inner_->cv.wait(lock);
+    }
+  }
 };
 
 /// A CoordinatorWorker owns Raft<CoordinatorRsm> instances. receives messages from the MachineManager.
@@ -129,9 +146,7 @@ class CoordinatorWorker {
 
  public:
   CoordinatorWorker(io::Io<IoImpl> io, Queue queue, Coordinator coordinator)
-      : io_(std::move(io)),
-        queue_(std::move(queue)),
-        coordinator_{std::move(io_.ForkLocal()), {}, std::move(coordinator)} {}
+      : io_(std::move(io)), queue_(std::move(queue)), coordinator_{std::move(io_), {}, std::move(coordinator)} {}
 
   CoordinatorWorker(CoordinatorWorker &&) noexcept = default;
   CoordinatorWorker &operator=(CoordinatorWorker &&) noexcept = default;
@@ -140,15 +155,12 @@ class CoordinatorWorker {
   ~CoordinatorWorker() = default;
 
   void Run() {
-    while (true) {
+    bool should_continue = true;
+    while (should_continue) {
       Message message = queue_.Pop();
 
-      const bool should_continue = std::visit(
-          [this](auto &&msg) { return this->Process(std::forward<decltype(msg)>(msg)); }, std::move(message));
-
-      if (!should_continue) {
-        return;
-      }
+      should_continue = std::visit([this](auto &&msg) { return this->Process(std::forward<decltype(msg)>(msg)); },
+                                   std::move(message));
     }
   }
 };
diff --git a/src/coordinator/shard_map.cpp b/src/coordinator/shard_map.cpp
index d0b2cf9ad..ea167db87 100644
--- a/src/coordinator/shard_map.cpp
+++ b/src/coordinator/shard_map.cpp
@@ -228,7 +228,7 @@ Hlc ShardMap::IncrementShardMapVersion() noexcept {
   return shard_map_version;
 }
 
-// TODO(antaljanosbenjamin) use a single map for all name id 
+// TODO(antaljanosbenjamin) use a single map for all name id
 // mapping and a single counter to maintain the next id
 std::unordered_map<uint64_t, std::string> ShardMap::IdToNames() {
   std::unordered_map<uint64_t, std::string> id_to_names;
@@ -248,6 +248,25 @@ std::unordered_map<uint64_t, std::string> ShardMap::IdToNames() {
 
 Hlc ShardMap::GetHlc() const noexcept { return shard_map_version; }
 
+boost::uuids::uuid NewShardUuid(uint64_t shard_id) {
+  return boost::uuids::uuid{0,
+                            0,
+                            0,
+                            0,
+                            0,
+                            0,
+                            0,
+                            0,
+                            static_cast<unsigned char>(shard_id >> 56U),
+                            static_cast<unsigned char>(shard_id >> 48U),
+                            static_cast<unsigned char>(shard_id >> 40U),
+                            static_cast<unsigned char>(shard_id >> 32U),
+                            static_cast<unsigned char>(shard_id >> 24U),
+                            static_cast<unsigned char>(shard_id >> 16U),
+                            static_cast<unsigned char>(shard_id >> 8U),
+                            static_cast<unsigned char>(shard_id)};
+}
+
 std::vector<ShardToInitialize> ShardMap::AssignShards(Address storage_manager,
                                                       std::set<boost::uuids::uuid> initialized) {
   std::vector<ShardToInitialize> ret{};
@@ -268,6 +287,7 @@ std::vector<ShardToInitialize> ShardMap::AssignShards(Address storage_manager,
         if (initialized.contains(aas.address.unique_id)) {
           machine_contains_shard = true;
           if (aas.status != Status::CONSENSUS_PARTICIPANT) {
+            mutated = true;
             spdlog::info("marking shard as full consensus participant: {}", aas.address.unique_id);
             aas.status = Status::CONSENSUS_PARTICIPANT;
           }
@@ -292,10 +312,13 @@ std::vector<ShardToInitialize> ShardMap::AssignShards(Address storage_manager,
       }
 
       if (!machine_contains_shard && shard.size() < label_space.replication_factor) {
+        // increment version for each new uuid for deterministic creation
+        IncrementShardMapVersion();
+
         Address address = storage_manager;
 
         // TODO(tyler) use deterministic UUID so that coordinators don't diverge here
-        address.unique_id = boost::uuids::uuid{boost::uuids::random_generator()()},
+        address.unique_id = NewShardUuid(shard_map_version.logical_id);
 
         spdlog::info("assigning shard manager to shard");
 
@@ -383,6 +406,7 @@ std::optional<LabelId> ShardMap::InitializeNewLabel(std::string label_name, std:
 void ShardMap::AddServer(Address server_address) {
   // Find a random place for the server to plug in
 }
+
 std::optional<LabelId> ShardMap::GetLabelId(const std::string &label) const {
   if (const auto it = labels.find(label); it != labels.end()) {
     return it->second;
diff --git a/src/io/address.hpp b/src/io/address.hpp
index a6e372edb..4e5cb7627 100644
--- a/src/io/address.hpp
+++ b/src/io/address.hpp
@@ -20,6 +20,8 @@
 #include <boost/uuid/uuid_generators.hpp>
 #include <boost/uuid/uuid_io.hpp>
 
+#include "utils/logging.hpp"
+
 namespace memgraph::io {
 
 struct PartialAddress {
@@ -58,18 +60,39 @@ struct Address {
   uint16_t last_known_port;
 
   static Address TestAddress(uint16_t port) {
+    MG_ASSERT(port <= 255);
+
     return Address{
-        .unique_id = boost::uuids::uuid{boost::uuids::random_generator()()},
+        .unique_id = boost::uuids::uuid{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, static_cast<unsigned char>(port)},
         .last_known_port = port,
     };
   }
 
+  // NB: don't use this in test code because it is non-deterministic
   static Address UniqueLocalAddress() {
     return Address{
         .unique_id = boost::uuids::uuid{boost::uuids::random_generator()()},
     };
   }
 
+  /// `Coordinator`s have constant UUIDs because there is at most one per ip/port pair.
+  Address ForkLocalCoordinator() {
+    return Address{
+        .unique_id = boost::uuids::uuid{1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0},
+        .last_known_ip = last_known_ip,
+        .last_known_port = last_known_port,
+    };
+  }
+
+  /// `ShardManager`s have constant UUIDs because there is at most one per ip/port pair.
+  Address ForkLocalShardManager() {
+    return Address{
+        .unique_id = boost::uuids::uuid{2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0},
+        .last_known_ip = last_known_ip,
+        .last_known_port = last_known_port,
+    };
+  }
+
   /// Returns a new ID with the same IP and port but a unique UUID.
   Address ForkUniqueAddress() {
     return Address{
diff --git a/src/io/future.hpp b/src/io/future.hpp
index bb2ef548c..98437b496 100644
--- a/src/io/future.hpp
+++ b/src/io/future.hpp
@@ -64,7 +64,6 @@ class Shared {
     waiting_ = true;
 
     while (!item_) {
-      bool simulator_progressed = false;
       if (simulator_notifier_) [[unlikely]] {
         // We can't hold our own lock while notifying
         // the simulator because notifying the simulator
@@ -77,7 +76,7 @@ class Shared {
         // so we have to get out of its way to avoid
         // a cyclical deadlock.
         lock.unlock();
-        simulator_progressed = std::invoke(simulator_notifier_);
+        std::invoke(simulator_notifier_);
         lock.lock();
         if (item_) {
           // item may have been filled while we
@@ -85,8 +84,7 @@ class Shared {
           // the simulator of our waiting_ status.
           break;
         }
-      }
-      if (!simulator_progressed) [[likely]] {
+      } else {
         cv_.wait(lock);
       }
       MG_ASSERT(!consumed_, "Future consumed twice!");
diff --git a/src/io/message_histogram_collector.hpp b/src/io/message_histogram_collector.hpp
index e663be988..a2cf266b7 100644
--- a/src/io/message_histogram_collector.hpp
+++ b/src/io/message_histogram_collector.hpp
@@ -13,6 +13,7 @@
 
 #include <chrono>
 #include <cmath>
+#include <compare>
 #include <unordered_map>
 
 #include <boost/core/demangle.hpp>
@@ -39,6 +40,8 @@ struct LatencyHistogramSummary {
   Duration p100;
   Duration sum;
 
+  friend bool operator==(const LatencyHistogramSummary &lhs, const LatencyHistogramSummary &rhs) = default;
+
   friend std::ostream &operator<<(std::ostream &in, const LatencyHistogramSummary &histo) {
     in << "{ \"count\": " << histo.count;
     in << ", \"p0\": " << histo.p0.count();
@@ -80,6 +83,8 @@ struct LatencyHistogramSummaries {
     return output;
   }
 
+  friend bool operator==(const LatencyHistogramSummaries &lhs, const LatencyHistogramSummaries &rhs) = default;
+
   friend std::ostream &operator<<(std::ostream &in, const LatencyHistogramSummaries &histo) {
     using memgraph::utils::print_helpers::operator<<;
     in << histo.latencies;
diff --git a/src/io/rsm/raft.hpp b/src/io/rsm/raft.hpp
index eccbf031b..4ec22ff87 100644
--- a/src/io/rsm/raft.hpp
+++ b/src/io/rsm/raft.hpp
@@ -19,7 +19,6 @@
 #include <map>
 #include <set>
 #include <thread>
-#include <unordered_map>
 #include <vector>
 
 #include <boost/core/demangle.hpp>
@@ -182,7 +181,7 @@ struct PendingClientRequest {
 
 struct Leader {
   std::map<Address, FollowerTracker> followers;
-  std::unordered_map<LogIndex, PendingClientRequest> pending_client_requests;
+  std::map<LogIndex, PendingClientRequest> pending_client_requests;
   Time last_broadcast = Time::min();
 
   std::string static ToString() { return "\tLeader   \t"; }
@@ -683,7 +682,7 @@ class Raft {
 
       return Leader{
           .followers = std::move(followers),
-          .pending_client_requests = std::unordered_map<LogIndex, PendingClientRequest>(),
+          .pending_client_requests = std::map<LogIndex, PendingClientRequest>(),
       };
     }
 
diff --git a/src/io/simulator/simulator_handle.cpp b/src/io/simulator/simulator_handle.cpp
index 74925812e..cc0bd0598 100644
--- a/src/io/simulator/simulator_handle.cpp
+++ b/src/io/simulator/simulator_handle.cpp
@@ -23,6 +23,12 @@ namespace memgraph::io::simulator {
 void SimulatorHandle::ShutDown() {
   std::unique_lock<std::mutex> lock(mu_);
   should_shut_down_ = true;
+  for (auto it = promises_.begin(); it != promises_.end();) {
+    auto &[promise_key, dop] = *it;
+    std::move(dop).promise.TimeOut();
+    it = promises_.erase(it);
+  }
+  can_receive_.clear();
   cv_.notify_all();
 }
 
@@ -46,52 +52,84 @@ void SimulatorHandle::IncrementServerCountAndWaitForQuiescentState(Address addre
     const bool all_servers_blocked = blocked_servers == server_addresses_.size();
 
     if (all_servers_blocked) {
+      spdlog::trace("quiescent state detected - {} out of {} servers now blocked on receive", blocked_servers,
+                    server_addresses_.size());
       return;
     }
 
+    spdlog::trace("not returning from quiescent because we see {} blocked out of {}", blocked_servers,
+                  server_addresses_.size());
     cv_.wait(lock);
   }
 }
 
+bool SortInFlight(const std::pair<Address, OpaqueMessage> &lhs, const std::pair<Address, OpaqueMessage> &rhs) {
+  // NB: never sort based on the request ID etc...
+  // This should only be used from std::stable_sort
+  // because by comparing on the from_address alone,
+  // we expect the sender ordering to remain
+  // deterministic.
+  const auto &[addr_1, om_1] = lhs;
+  const auto &[addr_2, om_2] = rhs;
+  return om_1.from_address < om_2.from_address;
+}
+
 bool SimulatorHandle::MaybeTickSimulator() {
   std::unique_lock<std::mutex> lock(mu_);
 
   const size_t blocked_servers = blocked_on_receive_.size();
 
-  if (blocked_servers < server_addresses_.size()) {
+  if (should_shut_down_ || blocked_servers < server_addresses_.size()) {
     // we only need to advance the simulator when all
     // servers have reached a quiescent state, blocked
     // on their own futures or receive methods.
     return false;
   }
 
+  // We allow the simulator to progress the state of the system only
+  // after all servers are blocked on receive.
+  spdlog::trace("~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ simulator tick ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~");
   stats_.simulator_ticks++;
-
+  blocked_on_receive_.clear();
   cv_.notify_all();
 
-  TimeoutPromisesPastDeadline();
+  bool timed_anything_out = TimeoutPromisesPastDeadline();
+
+  if (timed_anything_out) {
+    spdlog::trace("simulator progressing: timed out a request");
+  }
+
+  const Duration clock_advance = std::chrono::microseconds{time_distrib_(rng_)};
+
+  // We don't always want to advance the clock with every message that we deliver because
+  // when we advance it for every message, it causes timeouts to occur for any "request path"
+  // over a certain length. Alternatively, we don't want to simply deliver multiple messages
+  // in a single simulator tick because that would reduce the amount of concurrent message
+  // mixing that may naturally occur in production. This approach is to mod the random clock
+  // advance by a prime number (hopefully avoiding most harmonic effects that would be introduced
+  // by only advancing the clock by an even amount etc...) and only advancing the clock close to
+  // half of the time.
+  if (clock_advance.count() % 97 > 49) {
+    spdlog::trace("simulator progressing: clock advanced by {}", clock_advance.count());
+    cluster_wide_time_microseconds_ += clock_advance;
+    stats_.elapsed_time = cluster_wide_time_microseconds_ - config_.start_time;
+  }
+
+  if (cluster_wide_time_microseconds_ >= config_.abort_time) {
+    spdlog::error(
+        "Cluster has executed beyond its configured abort_time, and something may be failing to make progress "
+        "in an expected amount of time. The SimulatorConfig.rng_seed for this run is {}",
+        config_.rng_seed);
+    throw utils::BasicException{"Cluster has executed beyond its configured abort_time"};
+  }
 
   if (in_flight_.empty()) {
-    // return early here because there are no messages to schedule
-
-    // We tick the clock forward when all servers are blocked but
-    // there are no in-flight messages to schedule delivery of.
-    const Duration clock_advance = std::chrono::microseconds{time_distrib_(rng_)};
-    cluster_wide_time_microseconds_ += clock_advance;
-
-    if (cluster_wide_time_microseconds_ >= config_.abort_time) {
-      if (should_shut_down_) {
-        return false;
-      }
-      spdlog::error(
-          "Cluster has executed beyond its configured abort_time, and something may be failing to make progress "
-          "in an expected amount of time.");
-      throw utils::BasicException{"Cluster has executed beyond its configured abort_time"};
-    }
     return true;
   }
 
-  if (config_.scramble_messages) {
+  std::stable_sort(in_flight_.begin(), in_flight_.end(), SortInFlight);
+
+  if (config_.scramble_messages && in_flight_.size() > 1) {
     // scramble messages
     std::uniform_int_distribution<size_t> swap_distrib(0, in_flight_.size() - 1);
     const size_t swap_index = swap_distrib(rng_);
@@ -120,17 +158,22 @@ bool SimulatorHandle::MaybeTickSimulator() {
     if (should_drop || normal_timeout) {
       stats_.timed_out_requests++;
       dop.promise.TimeOut();
+      spdlog::trace("simulator timing out request ");
     } else {
       stats_.total_responses++;
       Duration response_latency = cluster_wide_time_microseconds_ - dop.requested_at;
       auto type_info = opaque_message.type_info;
       dop.promise.Fill(std::move(opaque_message), response_latency);
       histograms_.Measure(type_info, response_latency);
+      spdlog::trace("simulator replying to request");
     }
   } else if (should_drop) {
     // don't add it anywhere, let it drop
+    spdlog::trace("simulator silently dropping request");
   } else {
     // add to can_receive_ if not
+    spdlog::trace("simulator adding message to can_receive_ from {} to {}", opaque_message.from_address.last_known_port,
+                  opaque_message.to_address.last_known_port);
     const auto &[om_vec, inserted] =
         can_receive_.try_emplace(to_address.ToPartialAddress(), std::vector<OpaqueMessage>());
     om_vec->second.emplace_back(std::move(opaque_message));
diff --git a/src/io/simulator/simulator_handle.hpp b/src/io/simulator/simulator_handle.hpp
index 3420786c7..5a5ad1ec0 100644
--- a/src/io/simulator/simulator_handle.hpp
+++ b/src/io/simulator/simulator_handle.hpp
@@ -52,26 +52,29 @@ class SimulatorHandle {
   std::set<Address> blocked_on_receive_;
   std::set<Address> server_addresses_;
   std::mt19937 rng_;
-  std::uniform_int_distribution<int> time_distrib_{5, 50};
+  std::uniform_int_distribution<int> time_distrib_{0, 30000};
   std::uniform_int_distribution<int> drop_distrib_{0, 99};
   SimulatorConfig config_;
   MessageHistogramCollector histograms_;
   RequestId request_id_counter_{0};
 
-  void TimeoutPromisesPastDeadline() {
+  bool TimeoutPromisesPastDeadline() {
+    bool timed_anything_out = false;
     const Time now = cluster_wide_time_microseconds_;
     for (auto it = promises_.begin(); it != promises_.end();) {
       auto &[promise_key, dop] = *it;
       if (dop.deadline < now && config_.perform_timeouts) {
-        spdlog::info("timing out request from requester {}.", promise_key.requester_address.ToString());
+        spdlog::trace("timing out request from requester {}.", promise_key.requester_address.ToString());
         std::move(dop).promise.TimeOut();
         it = promises_.erase(it);
 
         stats_.timed_out_requests++;
+        timed_anything_out = true;
       } else {
         ++it;
       }
     }
+    return timed_anything_out;
   }
 
  public:
@@ -103,6 +106,7 @@ class SimulatorHandle {
   template <Message Request, Message Response>
   ResponseFuture<Response> SubmitRequest(Address to_address, Address from_address, Request &&request, Duration timeout,
                                          std::function<bool()> &&maybe_tick_simulator) {
+    spdlog::trace("submitting request to {}", to_address.last_known_port);
     auto type_info = TypeInfoFor(request);
 
     auto [future, promise] = memgraph::io::FuturePromisePairWithNotifier<ResponseResult<Response>>(
@@ -146,8 +150,6 @@ class SimulatorHandle {
   requires(sizeof...(Ms) > 0) RequestResult<Ms...> Receive(const Address &receiver, Duration timeout) {
     std::unique_lock<std::mutex> lock(mu_);
 
-    blocked_on_receive_.emplace(receiver);
-
     const Time deadline = cluster_wide_time_microseconds_ + timeout;
 
     auto partial_address = receiver.ToPartialAddress();
@@ -164,38 +166,40 @@ class SimulatorHandle {
           auto m_opt = std::move(message).Take<Ms...>();
           MG_ASSERT(m_opt.has_value(), "Wrong message type received compared to the expected type");
 
-          blocked_on_receive_.erase(receiver);
-
           return std::move(m_opt).value();
         }
       }
 
-      lock.unlock();
-      bool made_progress = MaybeTickSimulator();
-      lock.lock();
-      if (!should_shut_down_ && !made_progress) {
+      if (!should_shut_down_) {
+        if (!blocked_on_receive_.contains(receiver)) {
+          blocked_on_receive_.emplace(receiver);
+          spdlog::trace("blocking receiver {}", receiver.ToPartialAddress().port);
+          cv_.notify_all();
+        }
         cv_.wait(lock);
       }
     }
-
-    blocked_on_receive_.erase(receiver);
+    spdlog::trace("timing out receiver {}", receiver.ToPartialAddress().port);
 
     return TimedOut{};
   }
 
   template <Message M>
   void Send(Address to_address, Address from_address, RequestId request_id, M message) {
+    spdlog::trace("sending message from {} to {}", from_address.last_known_port, to_address.last_known_port);
     auto type_info = TypeInfoFor(message);
-    std::unique_lock<std::mutex> lock(mu_);
-    std::any message_any(std::move(message));
-    OpaqueMessage om{.to_address = to_address,
-                     .from_address = from_address,
-                     .request_id = request_id,
-                     .message = std::move(message_any),
-                     .type_info = type_info};
-    in_flight_.emplace_back(std::make_pair(std::move(to_address), std::move(om)));
+    {
+      std::unique_lock<std::mutex> lock(mu_);
+      std::any message_any(std::move(message));
+      OpaqueMessage om{.to_address = to_address,
+                       .from_address = from_address,
+                       .request_id = request_id,
+                       .message = std::move(message_any),
+                       .type_info = type_info};
+      in_flight_.emplace_back(std::make_pair(std::move(to_address), std::move(om)));
 
-    stats_.total_messages++;
+      stats_.total_messages++;
+    }  // lock dropped before cv notification
 
     cv_.notify_all();
   }
diff --git a/src/io/simulator/simulator_stats.hpp b/src/io/simulator/simulator_stats.hpp
index 7f529a456..1ae52ef28 100644
--- a/src/io/simulator/simulator_stats.hpp
+++ b/src/io/simulator/simulator_stats.hpp
@@ -13,6 +13,10 @@
 
 #include <cstdint>
 
+#include <fmt/format.h>
+
+#include "io/time.hpp"
+
 namespace memgraph::io::simulator {
 struct SimulatorStats {
   uint64_t total_messages = 0;
@@ -21,5 +25,22 @@ struct SimulatorStats {
   uint64_t total_requests = 0;
   uint64_t total_responses = 0;
   uint64_t simulator_ticks = 0;
+  Duration elapsed_time;
+
+  friend bool operator==(const SimulatorStats & /* lhs */, const SimulatorStats & /* rhs */) = default;
+
+  friend std::ostream &operator<<(std::ostream &in, const SimulatorStats &stats) {
+    auto elapsed_ms = std::chrono::duration_cast<std::chrono::milliseconds>(stats.elapsed_time).count();
+
+    std::string formated = fmt::format(
+        "SimulatorStats {{ total_messages: {}, dropped_messages: {}, timed_out_requests: {}, total_requests: {}, "
+        "total_responses: {}, simulator_ticks: {}, elapsed_time: {}ms }}",
+        stats.total_messages, stats.dropped_messages, stats.timed_out_requests, stats.total_requests,
+        stats.total_responses, stats.simulator_ticks, elapsed_ms);
+
+    in << formated;
+
+    return in;
+  }
 };
 };  // namespace memgraph::io::simulator
diff --git a/src/io/transport.hpp b/src/io/transport.hpp
index 2abf10af2..4994cb436 100644
--- a/src/io/transport.hpp
+++ b/src/io/transport.hpp
@@ -137,7 +137,14 @@ class Io {
   Address GetAddress() { return address_; }
   void SetAddress(Address address) { address_ = address; }
 
-  Io<I> ForkLocal() { return Io(implementation_, address_.ForkUniqueAddress()); }
+  Io<I> ForkLocal(boost::uuids::uuid uuid) {
+    Address new_address{
+        .unique_id = uuid,
+        .last_known_ip = address_.last_known_ip,
+        .last_known_port = address_.last_known_port,
+    };
+    return Io(implementation_, new_address);
+  }
 
   LatencyHistogramSummaries ResponseLatencies() { return implementation_.ResponseLatencies(); }
 };
diff --git a/src/machine_manager/machine_config.hpp b/src/machine_manager/machine_config.hpp
index 52711642b..8e8f503d7 100644
--- a/src/machine_manager/machine_config.hpp
+++ b/src/machine_manager/machine_config.hpp
@@ -42,6 +42,7 @@ struct MachineConfig {
   boost::asio::ip::address listen_ip;
   uint16_t listen_port;
   size_t shard_worker_threads = std::max(static_cast<unsigned int>(1), std::thread::hardware_concurrency());
+  bool sync_message_handling = false;
 };
 
 }  // namespace memgraph::machine_manager
diff --git a/src/machine_manager/machine_manager.hpp b/src/machine_manager/machine_manager.hpp
index f9ea8ff2a..04d5a3444 100644
--- a/src/machine_manager/machine_manager.hpp
+++ b/src/machine_manager/machine_manager.hpp
@@ -78,10 +78,10 @@ class MachineManager {
   MachineManager(io::Io<IoImpl> io, MachineConfig config, Coordinator coordinator)
       : io_(io),
         config_(config),
-        coordinator_address_(io.GetAddress().ForkUniqueAddress()),
-        shard_manager_{io.ForkLocal(), config.shard_worker_threads, coordinator_address_} {
-    auto coordinator_io = io.ForkLocal();
-    coordinator_io.SetAddress(coordinator_address_);
+        coordinator_address_(io.GetAddress().ForkLocalCoordinator()),
+        shard_manager_{io.ForkLocal(io.GetAddress().ForkLocalShardManager().unique_id), config.shard_worker_threads,
+                       coordinator_address_} {
+    auto coordinator_io = io.ForkLocal(coordinator_address_.unique_id);
     CoordinatorWorker coordinator_worker{coordinator_io, coordinator_queue_, coordinator};
     coordinator_handle_ = std::jthread([coordinator = std::move(coordinator_worker)]() mutable { coordinator.Run(); });
   }
@@ -101,11 +101,23 @@ class MachineManager {
   Address CoordinatorAddress() { return coordinator_address_; }
 
   void Run() {
-    while (!io_.ShouldShutDown()) {
+    while (true) {
+      MaybeBlockOnSyncHandling();
+
+      if (io_.ShouldShutDown()) {
+        break;
+      }
+
       const auto now = io_.Now();
 
+      uint64_t now_us = now.time_since_epoch().count();
+      uint64_t next_us = next_cron_.time_since_epoch().count();
+
       if (now >= next_cron_) {
+        spdlog::info("now {} >= next_cron_ {}", now_us, next_us);
         next_cron_ = Cron();
+      } else {
+        spdlog::info("now {} < next_cron_ {}", now_us, next_us);
       }
 
       Duration receive_timeout = std::max(next_cron_, now) - now;
@@ -194,10 +206,27 @@ class MachineManager {
   }
 
  private:
+  // This method exists for controlling concurrency
+  // during deterministic simulation testing.
+  void MaybeBlockOnSyncHandling() {
+    if (!config_.sync_message_handling) {
+      return;
+    }
+
+    // block on coordinator
+    coordinator_queue_.BlockOnQuiescence();
+
+    // block on shards
+    shard_manager_.BlockOnQuiescence();
+  }
+
   Time Cron() {
     spdlog::info("running MachineManager::Cron, address {}", io_.GetAddress().ToString());
     coordinator_queue_.Push(coordinator::coordinator_worker::Cron{});
-    return shard_manager_.Cron();
+    MaybeBlockOnSyncHandling();
+    Time ret = shard_manager_.Cron();
+    MaybeBlockOnSyncHandling();
+    return ret;
   }
 };
 
diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 94a41aee5..045d94709 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -799,7 +799,11 @@ InterpreterContext::InterpreterContext(storage::v3::Shard *db, const Interpreter
 
 Interpreter::Interpreter(InterpreterContext *interpreter_context) : interpreter_context_(interpreter_context) {
   MG_ASSERT(interpreter_context_, "Interpreter context must not be NULL");
-  auto query_io = interpreter_context_->io.ForkLocal();
+
+  // TODO(tyler) make this deterministic so that it can be tested.
+  auto random_uuid = boost::uuids::uuid{boost::uuids::random_generator()()};
+  auto query_io = interpreter_context_->io.ForkLocal(random_uuid);
+
   shard_request_manager_ = std::make_unique<msgs::ShardRequestManager<io::local_transport::LocalTransport>>(
       coordinator::CoordinatorClient<io::local_transport::LocalTransport>(
           query_io, interpreter_context_->coordinator_address, std::vector{interpreter_context_->coordinator_address}),
diff --git a/src/storage/v3/shard_manager.hpp b/src/storage/v3/shard_manager.hpp
index a3dc8b9b6..74ec8d2d1 100644
--- a/src/storage/v3/shard_manager.hpp
+++ b/src/storage/v3/shard_manager.hpp
@@ -190,6 +190,12 @@ class ShardManager {
                                      });
   }
 
+  void BlockOnQuiescence() {
+    for (const auto &worker : workers_) {
+      worker.BlockOnQuiescence();
+    }
+  }
+
  private:
   io::Io<IoImpl> io_;
   std::vector<shard_worker::Queue> workers_;
diff --git a/src/storage/v3/shard_worker.hpp b/src/storage/v3/shard_worker.hpp
index 46e02e6cc..547aa0a6f 100644
--- a/src/storage/v3/shard_worker.hpp
+++ b/src/storage/v3/shard_worker.hpp
@@ -80,6 +80,9 @@ struct QueueInner {
   // starvation by sometimes randomizing priorities, rather than following a strict
   // prioritization.
   std::deque<Message> queue;
+
+  uint64_t submitted = 0;
+  uint64_t calls_to_pop = 0;
 };
 
 /// There are two reasons to implement our own Queue instead of using
@@ -95,6 +98,8 @@ class Queue {
       MG_ASSERT(inner_.use_count() > 0);
       std::unique_lock<std::mutex> lock(inner_->mu);
 
+      inner_->submitted++;
+
       inner_->queue.emplace_back(std::forward<Message>(message));
     }  // lock dropped before notifying condition variable
 
@@ -105,6 +110,9 @@ class Queue {
     MG_ASSERT(inner_.use_count() > 0);
     std::unique_lock<std::mutex> lock(inner_->mu);
 
+    inner_->calls_to_pop++;
+    inner_->cv.notify_all();
+
     while (inner_->queue.empty()) {
       inner_->cv.wait(lock);
     }
@@ -114,6 +122,15 @@ class Queue {
 
     return message;
   }
+
+  void BlockOnQuiescence() const {
+    MG_ASSERT(inner_.use_count() > 0);
+    std::unique_lock<std::mutex> lock(inner_->mu);
+
+    while (inner_->calls_to_pop <= inner_->submitted) {
+      inner_->cv.wait(lock);
+    }
+  }
 };
 
 /// A ShardWorker owns Raft<ShardRsm> instances. receives messages from the ShardManager.
@@ -122,7 +139,6 @@ class ShardWorker {
   io::Io<IoImpl> io_;
   Queue queue_;
   std::priority_queue<std::pair<Time, uuid>, std::vector<std::pair<Time, uuid>>, std::greater<>> cron_schedule_;
-  Time next_cron_ = Time::min();
   std::map<uuid, ShardRaft<IoImpl>> rsm_map_;
 
   bool Process(ShutDown && /* shut_down */) { return false; }
@@ -175,10 +191,7 @@ class ShardWorker {
       return;
     }
 
-    auto rsm_io = io_.ForkLocal();
-    auto io_addr = rsm_io.GetAddress();
-    io_addr.unique_id = to_init.uuid;
-    rsm_io.SetAddress(io_addr);
+    auto rsm_io = io_.ForkLocal(to_init.uuid);
 
     // TODO(tyler) get peers from Coordinator in HeartbeatResponse
     std::vector<Address> rsm_peers = {};
@@ -208,15 +221,12 @@ class ShardWorker {
   ~ShardWorker() = default;
 
   void Run() {
-    while (true) {
+    bool should_continue = true;
+    while (should_continue) {
       Message message = queue_.Pop();
 
-      const bool should_continue =
+      should_continue =
           std::visit([&](auto &&msg) { return Process(std::forward<decltype(msg)>(msg)); }, std::move(message));
-
-      if (!should_continue) {
-        return;
-      }
     }
   }
 };
diff --git a/tests/simulation/basic_request.cpp b/tests/simulation/basic_request.cpp
index 1f6d60f77..868f4ac10 100644
--- a/tests/simulation/basic_request.cpp
+++ b/tests/simulation/basic_request.cpp
@@ -9,17 +9,25 @@
 // by the Apache License, Version 2.0, included in the file
 // licenses/APL.txt.
 
+#include <memory>
 #include <thread>
 
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+#include <spdlog/cfg/env.h>
+
+#include "io/message_histogram_collector.hpp"
 #include "io/simulator/simulator.hpp"
 #include "utils/print_helpers.hpp"
 
 using memgraph::io::Address;
 using memgraph::io::Io;
+using memgraph::io::LatencyHistogramSummaries;
 using memgraph::io::ResponseFuture;
 using memgraph::io::ResponseResult;
 using memgraph::io::simulator::Simulator;
 using memgraph::io::simulator::SimulatorConfig;
+using memgraph::io::simulator::SimulatorStats;
 using memgraph::io::simulator::SimulatorTransport;
 
 struct CounterRequest {
@@ -40,6 +48,7 @@ void run_server(Io<SimulatorTransport> io) {
       std::cout << "[SERVER] Error, continue" << std::endl;
       continue;
     }
+    std::cout << "[SERVER] Got message" << std::endl;
     auto request_envelope = request_result.GetValue();
     auto req = std::get<CounterRequest>(request_envelope.message);
 
@@ -50,13 +59,7 @@ void run_server(Io<SimulatorTransport> io) {
   }
 }
 
-int main() {
-  auto config = SimulatorConfig{
-      .drop_percent = 0,
-      .perform_timeouts = true,
-      .scramble_messages = true,
-      .rng_seed = 0,
-  };
+std::pair<SimulatorStats, LatencyHistogramSummaries> RunWorkload(SimulatorConfig &config) {
   auto simulator = Simulator(config);
 
   auto cli_addr = Address::TestAddress(1);
@@ -72,21 +75,47 @@ int main() {
     // send request
     CounterRequest cli_req;
     cli_req.proposal = i;
+    spdlog::info("[CLIENT] calling Request");
     auto res_f = cli_io.Request<CounterRequest, CounterResponse>(srv_addr, cli_req);
+    spdlog::info("[CLIENT] calling Wait");
     auto res_rez = std::move(res_f).Wait();
+    spdlog::info("[CLIENT] Wait returned");
     if (!res_rez.HasError()) {
-      std::cout << "[CLIENT] Got a valid response" << std::endl;
+      spdlog::info("[CLIENT] Got a valid response");
       auto env = res_rez.GetValue();
       MG_ASSERT(env.message.highest_seen == i);
-      std::cout << "response latency: " << env.response_latency.count() << " microseconds" << std::endl;
+      spdlog::info("response latency: {} microseconds", env.response_latency.count());
     } else {
-      std::cout << "[CLIENT] Got an error" << std::endl;
+      spdlog::info("[CLIENT] Got an error");
     }
   }
 
-  using memgraph::utils::print_helpers::operator<<;
-  std::cout << "response latencies: " << cli_io.ResponseLatencies() << std::endl;
-
   simulator.ShutDown();
+
+  return std::make_pair(simulator.Stats(), cli_io.ResponseLatencies());
+}
+
+int main() {
+  spdlog::cfg::load_env_levels();
+
+  auto config = SimulatorConfig{
+      .drop_percent = 0,
+      .perform_timeouts = true,
+      .scramble_messages = true,
+      .rng_seed = 0,
+  };
+
+  auto [sim_stats_1, latency_stats_1] = RunWorkload(config);
+  auto [sim_stats_2, latency_stats_2] = RunWorkload(config);
+
+  if (sim_stats_1 != sim_stats_2 || latency_stats_1 != latency_stats_2) {
+    spdlog::error("simulator stats diverged across runs");
+    spdlog::error("run 1 simulator stats: {}", sim_stats_1);
+    spdlog::error("run 2 simulator stats: {}", sim_stats_2);
+    spdlog::error("run 1 latency:\n{}", latency_stats_1.SummaryTable());
+    spdlog::error("run 2 latency:\n{}", latency_stats_2.SummaryTable());
+    std::terminate();
+  }
+
   return 0;
 }
diff --git a/tests/simulation/cluster_property_test.cpp b/tests/simulation/cluster_property_test.cpp
index 48327be5b..e74bad697 100644
--- a/tests/simulation/cluster_property_test.cpp
+++ b/tests/simulation/cluster_property_test.cpp
@@ -33,21 +33,31 @@ using io::Time;
 using io::simulator::SimulatorConfig;
 using storage::v3::kMaximumCronInterval;
 
-RC_GTEST_PROP(RandomClusterConfig, HappyPath, (ClusterConfig cluster_config, NonEmptyOpVec ops)) {
-  // TODO(tyler) set abort_time to something more restrictive than Time::max()
-
+RC_GTEST_PROP(RandomClusterConfig, HappyPath, (ClusterConfig cluster_config, NonEmptyOpVec ops, uint64_t rng_seed)) {
   spdlog::cfg::load_env_levels();
 
   SimulatorConfig sim_config{
       .drop_percent = 0,
       .perform_timeouts = false,
       .scramble_messages = true,
-      .rng_seed = 0,
+      .rng_seed = rng_seed,
       .start_time = Time::min(),
+      // TODO(tyler) set abort_time to something more restrictive than Time::max()
       .abort_time = Time::max(),
   };
 
-  RunClusterSimulation(sim_config, cluster_config, ops.ops);
+  auto [sim_stats_1, latency_stats_1] = RunClusterSimulation(sim_config, cluster_config, ops.ops);
+  auto [sim_stats_2, latency_stats_2] = RunClusterSimulation(sim_config, cluster_config, ops.ops);
+
+  if (latency_stats_1 != latency_stats_2) {
+    spdlog::error("simulator stats diverged across runs");
+    spdlog::error("run 1 simulator stats: {}", sim_stats_1);
+    spdlog::error("run 2 simulator stats: {}", sim_stats_2);
+    spdlog::error("run 1 latency:\n{}", latency_stats_1.SummaryTable());
+    spdlog::error("run 2 latency:\n{}", latency_stats_2.SummaryTable());
+    RC_ASSERT(latency_stats_1 == latency_stats_2);
+    RC_ASSERT(sim_stats_1 == sim_stats_2);
+  }
 }
 
 }  // namespace memgraph::tests::simulation
diff --git a/tests/simulation/raft.cpp b/tests/simulation/raft.cpp
index df619bd42..d725b0712 100644
--- a/tests/simulation/raft.cpp
+++ b/tests/simulation/raft.cpp
@@ -14,11 +14,15 @@
 #include <iostream>
 #include <map>
 #include <optional>
+#include <random>
 #include <set>
 #include <thread>
 #include <vector>
 
+#include <spdlog/cfg/env.h>
+
 #include "io/address.hpp"
+#include "io/message_histogram_collector.hpp"
 #include "io/rsm/raft.hpp"
 #include "io/rsm/rsm_client.hpp"
 #include "io/simulator/simulator.hpp"
@@ -27,6 +31,7 @@
 using memgraph::io::Address;
 using memgraph::io::Duration;
 using memgraph::io::Io;
+using memgraph::io::LatencyHistogramSummaries;
 using memgraph::io::ResponseEnvelope;
 using memgraph::io::ResponseFuture;
 using memgraph::io::ResponseResult;
@@ -123,16 +128,7 @@ void RunRaft(Raft<IoImpl, TestState, CasRequest, CasResponse, GetRequest, GetRes
   server.Run();
 }
 
-void RunSimulation() {
-  SimulatorConfig config{
-      .drop_percent = 5,
-      .perform_timeouts = true,
-      .scramble_messages = true,
-      .rng_seed = 0,
-      .start_time = Time::min() + std::chrono::microseconds{256 * 1024},
-      .abort_time = Time::max(),
-  };
-
+std::pair<SimulatorStats, LatencyHistogramSummaries> RunSimulation(SimulatorConfig &config) {
   auto simulator = Simulator(config);
 
   auto cli_addr = Address::TestAddress(1);
@@ -189,6 +185,7 @@ void RunSimulation() {
 
     auto write_cas_response_result = client.SendWriteRequest(cas_req);
     if (write_cas_response_result.HasError()) {
+      spdlog::debug("timed out");
       // timed out
       continue;
     }
@@ -229,6 +226,10 @@ void RunSimulation() {
 
   simulator.ShutDown();
 
+  srv_thread_1.join();
+  srv_thread_2.join();
+  srv_thread_3.join();
+
   SimulatorStats stats = simulator.Stats();
 
   spdlog::info("total messages:     {}", stats.total_messages);
@@ -240,21 +241,51 @@ void RunSimulation() {
 
   spdlog::info("========================== SUCCESS :) ==========================");
 
-  /*
-  this is implicit in jthread's dtor
-  srv_thread_1.join();
-  srv_thread_2.join();
-  srv_thread_3.join();
-  */
+  return std::make_pair(simulator.Stats(), cli_io.ResponseLatencies());
+}
+
+void RunWithSeed(uint64_t seed) {
+  SimulatorConfig config{
+      .drop_percent = 5,
+      .perform_timeouts = true,
+      .scramble_messages = true,
+      .rng_seed = seed,
+      .start_time = Time::min() + std::chrono::microseconds{256 * 1024},
+      .abort_time = Time::min() + std::chrono::seconds{3600},
+  };
+
+  spdlog::error("========================== NEW SIMULATION, replay with RunWithSeed({}) ==========================",
+                seed);
+  spdlog::info("\tTime\t\tTerm\tPort\tRole\t\tMessage\n");
+  auto [sim_stats_1, latency_stats_1] = RunSimulation(config);
+
+  spdlog::info("========================== NEW SIMULATION, replay with RunWithSeed({}) ==========================",
+               seed);
+  spdlog::info("\tTime\t\tTerm\tPort\tRole\t\tMessage\n");
+  auto [sim_stats_2, latency_stats_2] = RunSimulation(config);
+
+  if (sim_stats_1 != sim_stats_2 || latency_stats_1 != latency_stats_2) {
+    spdlog::error("simulator stats diverged across runs for test rng_seed: {}", seed);
+    spdlog::error("run 1 simulator stats: {}", sim_stats_1);
+    spdlog::error("run 2 simulator stats: {}", sim_stats_2);
+    spdlog::error("run 1 latency:\n{}", latency_stats_1.SummaryTable());
+    spdlog::error("run 2 latency:\n{}", latency_stats_2.SummaryTable());
+    std::terminate();
+  }
 }
 
 int main() {
+  spdlog::cfg::load_env_levels();
+
+  std::random_device random_device;
+  std::mt19937 generator(random_device());
+  std::uniform_int_distribution<> distribution;
+
   int n_tests = 50;
 
   for (int i = 0; i < n_tests; i++) {
-    spdlog::info("========================== NEW SIMULATION {} ==========================", i);
-    spdlog::info("\tTime\t\tTerm\tPort\tRole\t\tMessage\n");
-    RunSimulation();
+    uint64_t seed = distribution(generator);
+    RunWithSeed(seed);
   }
 
   spdlog::info("passed {} tests!", n_tests);
diff --git a/tests/simulation/shard_rsm.cpp b/tests/simulation/shard_rsm.cpp
index 64d0a0861..230bd9222 100644
--- a/tests/simulation/shard_rsm.cpp
+++ b/tests/simulation/shard_rsm.cpp
@@ -1115,11 +1115,12 @@ int TestMessages() {
   ConcreteShardRsm shard_server3(std::move(shard_server_io_3), address_for_3, ShardRsm(std::move(shard_ptr3)));
 
   auto server_thread1 = std::jthread([&shard_server1]() { shard_server1.Run(); });
-  auto server_thread2 = std::jthread([&shard_server2]() { shard_server2.Run(); });
-  auto server_thread3 = std::jthread([&shard_server3]() { shard_server3.Run(); });
-
   simulator.IncrementServerCountAndWaitForQuiescentState(shard_server_1_address);
+
+  auto server_thread2 = std::jthread([&shard_server2]() { shard_server2.Run(); });
   simulator.IncrementServerCountAndWaitForQuiescentState(shard_server_2_address);
+
+  auto server_thread3 = std::jthread([&shard_server3]() { shard_server3.Run(); });
   simulator.IncrementServerCountAndWaitForQuiescentState(shard_server_3_address);
 
   std::cout << "Beginning test after servers have become quiescent." << std::endl;
diff --git a/tests/simulation/test_cluster.hpp b/tests/simulation/test_cluster.hpp
index 6a32a391d..0ac7ca1da 100644
--- a/tests/simulation/test_cluster.hpp
+++ b/tests/simulation/test_cluster.hpp
@@ -24,6 +24,7 @@
 #include "coordinator/shard_map.hpp"
 #include "generated_operations.hpp"
 #include "io/address.hpp"
+#include "io/message_histogram_collector.hpp"
 #include "io/simulator/simulator.hpp"
 #include "io/simulator/simulator_config.hpp"
 #include "io/simulator/simulator_transport.hpp"
@@ -57,6 +58,7 @@ using io::simulator::SimulatorStats;
 using io::simulator::SimulatorTransport;
 using machine_manager::MachineConfig;
 using machine_manager::MachineManager;
+using memgraph::io::LatencyHistogramSummaries;
 using msgs::ReadRequests;
 using msgs::ReadResponses;
 using msgs::WriteRequests;
@@ -75,6 +77,8 @@ MachineManager<SimulatorTransport> MkMm(Simulator &simulator, std::vector<Addres
       .is_coordinator = true,
       .listen_ip = addr.last_known_ip,
       .listen_port = addr.last_known_port,
+      .shard_worker_threads = 4,
+      .sync_message_handling = true,
   };
 
   Io<SimulatorTransport> io = simulator.Register(addr);
@@ -210,17 +214,19 @@ struct DetachIfDropped {
   }
 };
 
-void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig &cluster_config,
-                          const std::vector<Op> &ops) {
+std::pair<SimulatorStats, LatencyHistogramSummaries> RunClusterSimulation(const SimulatorConfig &sim_config,
+                                                                          const ClusterConfig &cluster_config,
+                                                                          const std::vector<Op> &ops) {
   spdlog::info("========================== NEW SIMULATION ==========================");
 
   auto simulator = Simulator(sim_config);
 
-  auto cli_addr = Address::TestAddress(1);
-  auto machine_1_addr = cli_addr.ForkUniqueAddress();
+  auto machine_1_addr = Address::TestAddress(1);
+  auto cli_addr = Address::TestAddress(2);
+  auto cli_addr_2 = Address::TestAddress(3);
 
   Io<SimulatorTransport> cli_io = simulator.Register(cli_addr);
-  Io<SimulatorTransport> cli_io_2 = simulator.Register(Address::TestAddress(2));
+  Io<SimulatorTransport> cli_io_2 = simulator.Register(cli_addr_2);
 
   auto coordinator_addresses = std::vector{
       machine_1_addr,
@@ -232,6 +238,7 @@ void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig
   Address coordinator_address = mm_1.CoordinatorAddress();
 
   auto mm_thread_1 = std::jthread(RunMachine, std::move(mm_1));
+  simulator.IncrementServerCountAndWaitForQuiescentState(machine_1_addr);
 
   auto detach_on_error = DetachIfDropped{.handle = mm_thread_1};
 
@@ -257,6 +264,8 @@ void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig
 
   simulator.ShutDown();
 
+  mm_thread_1.join();
+
   SimulatorStats stats = simulator.Stats();
 
   spdlog::info("total messages:     {}", stats.total_messages);
@@ -268,10 +277,8 @@ void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig
 
   auto histo = cli_io_2.ResponseLatencies();
 
-  using memgraph::utils::print_helpers::operator<<;
-  std::cout << "response latencies: " << histo << std::endl;
-
   spdlog::info("========================== SUCCESS :) ==========================");
+  return std::make_pair(stats, histo);
 }
 
 }  // namespace memgraph::tests::simulation