From adab0a5282cace1b9e2c0a59fe94e99a80a27299 Mon Sep 17 00:00:00 2001
From: Roel Aaij <roel.aaij@cern.ch>
Date: Thu, 3 Dec 2020 15:48:46 +0100
Subject: [PATCH 001/120] Bring back AllenOnline

---
 AllenOnline/CMakeLists.txt             |  37 +++
 AllenOnline/options/Allen.py           |  71 +++++
 AllenOnline/src/AllenApplication.cpp   | 349 +++++++++++++++++++++++++
 AllenOnline/src/AllenApplication.h     |  96 +++++++
 AllenOnline/src/AllenConfiguration.cpp |  19 ++
 AllenOnline/src/AllenConfiguration.h   |  33 +++
 6 files changed, 605 insertions(+)
 create mode 100644 AllenOnline/CMakeLists.txt
 create mode 100755 AllenOnline/options/Allen.py
 create mode 100644 AllenOnline/src/AllenApplication.cpp
 create mode 100644 AllenOnline/src/AllenApplication.h
 create mode 100644 AllenOnline/src/AllenConfiguration.cpp
 create mode 100644 AllenOnline/src/AllenConfiguration.h

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
new file mode 100644
index 000000000..345c2d35d
--- /dev/null
+++ b/AllenOnline/CMakeLists.txt
@@ -0,0 +1,37 @@
+################################################################################
+# Package: AllenOnline
+################################################################################
+gaudi_subdir(AllenOnline v1r0)
+
+gaudi_depends_on_subdirs(Online/OnlineKernel
+                         Online/GaudiOnline
+                         Online/dim
+                         Online/RPC
+                         Tools/ZeroMQ
+                         Online/Parsers
+                         Dumpers/BinaryDumpers)
+
+include(AllenExport)
+
+find_package(cppgsl REQUIRED)
+find_package(ZMQ REQUIRED)
+find_package(Boost REQUIRED)
+find_package(ROOT REQUIRED)
+include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
+  ${ZMQ_INCLUDE_DIRS}
+  ${CPPGSL_INCLUDE_DIR}
+  ${ROOT_INCLUDE_DIRS})
+
+# find_package(MPI REQUIRED)
+# include_directories(SYSTEM ${MPI_CXX_INCLUDE_PATH})
+# add_compile_definitions(HAVE_MPI)
+
+include_directories(${CMAKE_SOURCE_DIR}/main/include)
+
+gaudi_add_module(AllenOnline
+                 src/*.cpp
+                 INCLUDE_DIRS OnlineBase ROOT Online/DIM GaudiKernel cppgsl Online/OnlineKernel Online/ZeroMQ
+                 LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQ ZMQLib RPC Parsers
+                                OnlineKernel AllenLib DAQEventLib DAQKernelLib GaudiAlgLib
+                                PrKernel VPDetLib UTDetLib UTKernelLib
+                                FTDetLib -lrt)
diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
new file mode 100755
index 000000000..2f9769595
--- /dev/null
+++ b/AllenOnline/options/Allen.py
@@ -0,0 +1,71 @@
+#!/usr/bin/env python2
+from Configurables import LHCbApp, CondDB, ApplicationMgr
+from Configurables import DumpUTGeometry, DumpFTGeometry, DumpMuonTable
+from Configurables import DumpMuonGeometry, DumpVPGeometry, AllenUpdater
+from Configurables import DumpMagneticField, DumpBeamline, DumpUTLookupTables
+from Configurables import AllenConfiguration
+from Configurables import MonitorSvc
+from Configurables import Online__Configuration as OnlineConfiguration
+
+app = LHCbApp(
+    DataType="Upgrade",
+    EvtMax=1000,
+    Simulation=True,
+    DDDBtag="dddb-20171122",
+    CondDBtag="sim-20180530-vc-md100")
+
+# Upgrade DBs
+CondDB().Upgrade = True
+
+producers = [
+    p(DumpToFile=False)
+    for p in (DumpVPGeometry, DumpUTGeometry, DumpFTGeometry, DumpMuonGeometry,
+              DumpMuonTable, DumpMagneticField, DumpBeamline,
+              DumpUTLookupTables)
+]
+
+online_conf = OnlineConfiguration("Application")
+online_conf.debug = False
+online_conf.classType = 1
+online_conf.automatic = False
+online_conf.monitorType = 'MonitorSvc'
+online_conf.logDeviceType = 'RTL::Logger::LogDevice'
+online_conf.logDeviceFormat = '%TIME%LEVEL%-8NODE: %-32PROCESS %-20SOURCE'
+online_conf.OutputLevel = 3
+online_conf.IOOutputLevel = 3
+
+allen_conf = AllenConfiguration()
+allen_conf.EventsPerSlice = 1000
+allen_conf.NonStop = True
+allen_conf.MPI = False
+allen_conf.Receivers = "mlx5_0:1"
+allen_conf.NThreads = 8
+allen_conf.NSlices = 16
+# allen_conf.Output = "tcp://192.168.1.101:35000"
+# allen_conf.Device = "01:00.0"
+allen_conf.Input = [
+    "/scratch/raaij/mep/upgrade_mc_minbias_scifi_v5_pf3000.mep"
+]
+allen_conf.Device = "0"
+allen_conf.OutputLevel = 2
+
+monSvc = MonitorSvc('MonitorSvc')
+monSvc.PartitionName = 'Allen'
+monSvc.ExpandNameInfix = '<proc>'
+monSvc.ExpandCounterServices = True
+monSvc.UniqueServiceNames = True
+
+# Add the services that will produce the non-event-data
+ApplicationMgr().ExtSvc += [
+    monSvc,
+    AllenUpdater(OutputLevel=2),
+] + producers
+
+# Some extra stuff for timing table
+ApplicationMgr().EvtSel = "NONE"
+ApplicationMgr().ExtSvc += [
+    'ToolSvc', 'AuditorSvc', 'AllenConfiguration',
+    'Online::Configuration/Application', 'ZeroMQSvc'
+]
+
+# gaudi = AppMgr()
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
new file mode 100644
index 000000000..a19c58f64
--- /dev/null
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -0,0 +1,349 @@
+#include <dlfcn.h>
+
+#include <iostream>
+#include <chrono>
+#include <thread>
+#include <cmath>
+#include <regex>
+
+#include <GaudiKernel/IJobOptionsSvc.h>
+#include <GaudiKernel/IMessageSvc.h>
+#include <GaudiKernel/IAppMgrUI.h>
+#include <GaudiKernel/IProperty.h>
+#include <GaudiKernel/ISvcLocator.h>
+#include <GaudiKernel/AppReturnCode.h>
+#include <Gaudi/Property.h>
+#include <GaudiKernel/SmartIF.h>
+#include <GaudiKernel/IMonitorSvc.h>
+#include <CPP/Event.h>
+#include <RTL/strdef.h>
+#include <RTL/rtl.h>
+#include <dim/dis.hxx>
+
+#include <GaudiOnline/OnlineApplication.h>
+
+#include <Allen/Allen.h>
+
+#ifdef HAVE_MPI
+#include <Allen/MPIConfig.h>
+#endif
+
+#include "AllenConfiguration.h"
+#include "AllenApplication.h"
+
+namespace {
+  using namespace std::string_literals;
+
+  std::string resolveEnvVars(std::string s)
+  {
+    std::regex envExpr {"\\$\\{([A-Za-z0-9_]+)\\}"};
+    std::smatch m;
+    while (std::regex_search(s, m, envExpr)) {
+      std::string rep;
+      System::getEnv(m[1].str(), rep);
+      s = s.replace(m[1].first - 2, m[1].second + 1, rep);
+    }
+    return s;
+  }
+} // namespace
+
+/// Factory instantiation
+DECLARE_COMPONENT(AllenApplication)
+
+/// Reset counters at start
+void AllenApplication::monitor_t::reset()
+{
+  mepsIn = 0;
+  eventsOut = 0;
+}
+
+/// Specialized constructor
+AllenApplication::AllenApplication(Options opts) : OnlineApplication(opts) {}
+
+// Default destructor
+AllenApplication::~AllenApplication()
+{
+  if (m_handle) {
+    dlclose(m_handle);
+  }
+}
+
+/// Stop the application                             (RUNNING    -> READY)
+int AllenApplication::stop()
+{
+  fireIncident("DAQ_CANCEL");
+
+  m_zmqSvc->send(*m_allenControl, "STOP");
+
+  zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
+  m_zmqSvc->poll(&items[0], 1, -1);
+  if (items[0].revents & zmq::POLLIN) {
+    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
+    if (msg == "READY") {
+      m_logger->info("Allen event loop is stopped");
+    }
+    else {
+      m_logger->error("Allen event loop failed to stop");
+      return Online::ONLINE_ERROR;
+    }
+  }
+
+  return OnlineApplication::stop();
+}
+
+/// Cancel the application: Cancel IO request/Event loop
+int AllenApplication::cancel() { return 1; }
+
+/// Internal: Initialize the application            (NOT_READY  -> READY)
+int AllenApplication::configureApplication()
+{
+  int ret = OnlineApplication::configureApplication();
+  if (ret != Online::ONLINE_OK) return ret;
+
+  // dlopen libAllenLib
+  m_handle = dlopen("libAllenLib.so", RTLD_LAZY);
+  if (!m_handle) {
+    m_logger->error("Failed to dlopen libAllenLib");
+    return Online::ONLINE_ERROR;
+  }
+
+  // reset errors
+  dlerror();
+  // load the symbol
+  m_allenFun = (allen_t) dlsym(m_handle, "allen");
+  const char* dlsym_error = dlerror();
+  if (dlsym_error) {
+    m_logger->error("Failed to get 'allen' from libAllenLib");
+    dlclose(m_handle);
+    return Online::ONLINE_ERROR;
+  }
+
+  SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
+
+  if (!m_config->monitorType.empty()) {
+
+    m_monMEPs.reset(new Service("MEPs", sloc));
+    m_monEvents.reset(new Service("Events", sloc));
+
+    m_monSvc = sloc->service<IMonitorSvc>(m_config->monitorType);
+    if (!m_monSvc.get()) {
+      m_logger->error("Cannot access monitoring service of type %s.", m_config->monitorType.c_str());
+      return Online::ONLINE_ERROR;
+    }
+    m_monSvc->declareInfo("IN", m_monitor.mepsIn, "Number of MEPs received for processing", m_monMEPs);
+    m_monSvc->declareInfo("OUT", m_monitor.mepsDone, "Number of MEPs fully processed", m_monMEPs);
+    m_monSvc->declareInfo("OUT", m_monitor.eventsOut, "Number of events fully output", m_monEvents);
+  }
+
+  auto config = sloc->service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
+  if (!config.get()) {
+    m_logger->throwError("Failed to retrieve AllenConfiguration.");
+    return Online::ONLINE_ERROR;
+  }
+  m_allenConfig = config.get();
+
+  m_zmqSvc = sloc->service<IZeroMQSvc>("ZeroMQSvc");
+  if (!m_zmqSvc) {
+    m_logger->error("Failed to retrieve IZeroMQSvc.");
+    return Online::ONLINE_ERROR;
+  }
+
+  SmartIF<IService> updater = sloc->service<IService>("AllenUpdater");
+  if (!updater.get()) {
+    m_logger->error("Failed to retrieve AllenUpdater.");
+    return Online::ONLINE_ERROR;
+  }
+  m_updater = dynamic_cast<Allen::NonEventData::IUpdater*>(updater.get());
+  if (updater == nullptr) {
+    m_logger->error("Failed to cast AllenUpdater");
+    return Online::ONLINE_ERROR;
+  }
+
+  if (m_allenConfig->withMPI.value()) {
+    auto success = initMPI();
+    if (!success) {
+      m_logger->error("Failed to initialize MPI");
+      return Online::ONLINE_ERROR;
+    }
+  }
+
+  m_allenControl = m_zmqSvc->socket(zmq::PAIR);
+  m_allenControl->bind(m_controlConnection.c_str());
+
+  m_allenThread = std::thread {&AllenApplication::allenLoop, this};
+
+  zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
+  m_zmqSvc->poll(&items[0], 1, -1);
+  if (items[0].revents & zmq::POLLIN) {
+    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
+    if (msg == "READY") {
+      m_logger->info("Allen event loop is ready");
+    }
+  }
+
+  return ret;
+}
+
+/// Internal: Finalize the application              (READY      -> NOT_READY)
+int AllenApplication::finalizeApplication()
+{
+  m_zmqSvc->send(*m_allenControl, "RESET");
+
+  zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
+  m_zmqSvc->poll(&items[0], 1, -1);
+  if (items[0].revents & zmq::POLLIN) {
+    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
+    if (msg == "NOT_READY") {
+      m_logger->info("Allen event loop has exited");
+
+      m_allenThread.join();
+    }
+    else {
+      m_logger->error("Allen event loop failed to exit");
+      return Online::ONLINE_ERROR;
+    }
+  }
+
+  if (m_monSvc.get()) {
+    m_monSvc->undeclareAll(m_monMEPs);
+    m_monSvc->undeclareAll(m_monEvents);
+    m_monSvc.reset();
+  }
+  m_monMEPs.reset();
+  m_monEvents.reset();
+  return OnlineApplication::finalizeApplication();
+}
+
+/// Internal: Start the application                 (READY      -> RUNNING)
+int AllenApplication::startApplication()
+{
+  StatusCode sc = app->start();
+  if (!sc.isSuccess()) {
+    return Online::ONLINE_ERROR;
+  }
+
+  m_zmqSvc->send(*m_allenControl, "START");
+
+  zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
+  m_zmqSvc->poll(&items[0], 1, -1);
+  if (items[0].revents & zmq::POLLIN) {
+    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
+    if (msg == "RUNNING") {
+      m_logger->info("Allen event loop is running");
+    }
+    else {
+      m_logger->error("Allen event loop failed to start");
+      return Online::ONLINE_ERROR;
+    }
+  }
+
+  fireIncident("DAQ_RUNNING");
+  fireIncident("APP_RUNNING");
+  return Online::ONLINE_OK;
+}
+
+/// Pause the application                            (RUNNING    -> READY)
+int AllenApplication::pauseProcessing()
+{
+  m_logger->debug("Pause the application.");
+  return OnlineApplication::pauseProcessing();
+}
+
+/// Continue the application                        (PAUSED -> RUNNING )
+int AllenApplication::continueProcessing()
+{
+  m_logger->debug("Resume application processing.");
+  return OnlineApplication::continueProcessing();
+}
+
+bool AllenApplication::initMPI()
+{
+#ifdef HAVE_MPI
+  // MPI initialization
+  auto len = name().length();
+  int provided = 0;
+  m_mpiArgv = new char*[1];
+  m_mpiArgv[0] = new char[len];
+  ::strncpy(m_mpiArgv[0], name().c_str(), len);
+  MPI_Init_thread(&m_mpiArgc, &m_mpiArgv, MPI_THREAD_MULTIPLE, &provided);
+  if (provided != MPI_THREAD_MULTIPLE) {
+    m_logger->error("Failed to initialize MPI multi thread support.");
+    return false;
+  }
+
+  // Communication size
+  int comm_size = 0;
+  MPI_Comm_size(MPI_COMM_WORLD, &comm_size);
+  if (comm_size > MPI::comm_size) {
+    std::string e = "This program requires at most "s + std::to_string(MPI::comm_size) + " processes.";
+    m_logger->error(e.c_str());
+    return false;
+  }
+
+  // MPI: Who am I?
+  MPI_Comm_rank(MPI_COMM_WORLD, &m_rank);
+
+  if (m_rank != MPI::receiver) {
+    m_logger->error("AllenApplication can only function as MPI receiver.");
+    return false;
+  }
+  else {
+    return true;
+  }
+#else
+  m_logger->error("MPI requested, but Allen was not built with MPI support.");
+  return false;
+#endif
+}
+
+void AllenApplication::allenLoop()
+{
+
+  auto json = resolveEnvVars(m_allenConfig->json);
+  auto paramDir = resolveEnvVars(m_allenConfig->paramDir);
+
+  //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
+  //--device 23:00.0
+  std::map<std::string, std::string> allen_options = {{"events-per-slice", std::to_string(m_allenConfig->eps.value())},
+                                                      {"non-stop", std::to_string(m_allenConfig->nonStop.value())},
+                                                      {"c", std::to_string(m_allenConfig->check.value())},
+                                                      {"v", std::to_string(6 - m_config->outputLevel())},
+                                                      {"t", std::to_string(m_allenConfig->nThreads.value())},
+                                                      {"geometry", paramDir},
+                                                      {"configuration", json},
+                                                      {"device", m_allenConfig->device.value()}};
+
+  if (!m_allenConfig->output.value().empty()) {
+    allen_options["output-file"] = m_allenConfig->output.value();
+  }
+
+  if (m_allenConfig->nSlices.value() != 0) {
+    allen_options["s"] = std::to_string(m_allenConfig->nSlices.value());
+  }
+
+  auto const& input = m_allenConfig->input.value();
+  if (m_allenConfig->withMPI.value() == true) {
+    if (!m_allenConfig->receivers.value().empty()) {
+      allen_options["with-mpi"] = m_allenConfig->receivers.value();
+    }
+    else {
+      allen_options["with-mpi"] = "1";
+    }
+  }
+  else if (input.empty()) {
+    m_logger->throwError("No input files specified");
+  }
+  else {
+    std::stringstream ss;
+    bool mep = false;
+    for (size_t i = 0; i < input.size(); ++i) {
+      if (i != 0) ss << ",";
+      if (input[i].find(".mep") != std::string::npos) mep = true;
+      ss << input[i];
+    }
+    auto files = ss.str();
+    allen_options[(mep ? "mep" : "mdf")] = files;
+  }
+
+  m_allenFun(allen_options, m_updater, m_zmqSvc.get(), m_controlConnection);
+}
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
new file mode 100644
index 000000000..907626573
--- /dev/null
+++ b/AllenOnline/src/AllenApplication.h
@@ -0,0 +1,96 @@
+#include <dlfcn.h>
+
+#include <iostream>
+#include <chrono>
+#include <cmath>
+
+#include <GaudiKernel/IJobOptionsSvc.h>
+#include <GaudiKernel/IMessageSvc.h>
+#include <GaudiKernel/IAppMgrUI.h>
+#include <GaudiKernel/IProperty.h>
+#include <GaudiKernel/ISvcLocator.h>
+#include <GaudiKernel/AppReturnCode.h>
+#include <Gaudi/Property.h>
+#include <GaudiKernel/SmartIF.h>
+#include <GaudiKernel/IMonitorSvc.h>
+#include <CPP/Event.h>
+#include <RTL/strdef.h>
+#include <RTL/rtl.h>
+#include <dim/dis.hxx>
+
+#include <GaudiOnline/OnlineApplication.h>
+#include <ZeroMQ/IZeroMQSvc.h>
+
+#include <Allen/Allen.h>
+
+#include "AllenConfiguration.h"
+
+class AllenApplication : public Online::OnlineApplication {
+public:
+  /// Structurte containing all monitoring items
+  struct monitor_t {
+    long mepsIn = 0;
+    long mepsDone = 0;
+    long eventsOut = 0;
+    monitor_t() = default;
+    virtual ~monitor_t() = default;
+    void reset();
+  } m_monitor;
+
+  // Specialized constructor
+  AllenApplication(Options opts);
+  // Default destructor
+  virtual ~AllenApplication();
+
+  /// Cancel the application: Cancel IO request/Event loop
+  int cancel() override;
+
+  /// Internal: Initialize the application            (NOT_READY  -> READY)
+  int configureApplication() override;
+  /// Internal: Finalize the application              (READY      -> NOT_READY)
+  int finalizeApplication() override;
+
+  /// Internal: Start the application                 (READY      -> RUNNING)
+  int startApplication() override;
+  /// Stop the application                            (RUNNING    -> READY)
+  int stop() override;
+  /// Pause the application                           (RUNNING    -> PAUSED)
+  int pauseProcessing() override;
+  /// Continue the application                        (PAUSED -> RUNNING )
+  int continueProcessing() override;
+
+  // Main function running the Allen event loop
+  void allenLoop();
+
+  bool initMPI();
+
+private:
+  /// Reference to the monitoring service
+  SmartIF<IMonitorSvc> m_monSvc;
+
+  /// Handles to helper service to properly name burst counters
+  SmartIF<IService> m_monMEPs;
+  /// Handles to helper service to properly name event counters
+  SmartIF<IService> m_monEvents;
+
+  // ZeroMQSvc
+  SmartIF<IZeroMQSvc> m_zmqSvc;
+
+  Allen::NonEventData::IUpdater* m_updater = nullptr;
+  AllenConfiguration const* m_allenConfig = nullptr;
+
+  std::string m_controlConnection = "inproc://AllenApplicationControl";
+
+  // dlopen stuff to workaround segfault in genconf.exe
+  void* m_handle = nullptr;
+  typedef int (
+    *allen_t)(std::map<std::string, std::string>, Allen::NonEventData::IUpdater*, IZeroMQSvc* zmqSvc, std::string_view);
+  allen_t m_allenFun = nullptr;
+
+  std::thread m_allenThread;
+  std::optional<zmq::socket_t> m_allenControl;
+
+  char** m_mpiArgv = nullptr;
+  int m_mpiArgc = 1;
+  int m_rank = -1;
+};
diff --git a/AllenOnline/src/AllenConfiguration.cpp b/AllenOnline/src/AllenConfiguration.cpp
new file mode 100644
index 000000000..8d3af7610
--- /dev/null
+++ b/AllenOnline/src/AllenConfiguration.cpp
@@ -0,0 +1,19 @@
+#include "GaudiKernel/Service.h"
+#include "AllenConfiguration.h"
+
+DECLARE_COMPONENT(AllenConfiguration)
+
+/// Query interfaces of Interface
+StatusCode AllenConfiguration::queryInterface(const InterfaceID& riid, void** ppv)
+{
+  if (AllenConfiguration::interfaceID().versionMatch(riid)) {
+    *ppv = this;
+    addRef();
+    return StatusCode::SUCCESS;
+  }
+  return Service::queryInterface(riid, ppv);
+}
+
+AllenConfiguration::AllenConfiguration(std::string name, ISvcLocator* svcloc) : Service(name, svcloc) {}
+
+AllenConfiguration::~AllenConfiguration() {}
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
new file mode 100644
index 000000000..4a7a431ce
--- /dev/null
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -0,0 +1,33 @@
+#pragma once
+
+#include "GaudiKernel/Service.h"
+
+class AllenConfiguration : public Service {
+public:
+  /// Retrieve interface ID
+  static const InterfaceID& interfaceID()
+  {
+    // Declaration of the interface ID.
+    static const InterfaceID iid("AllenConfiguration", 0, 0);
+    return iid;
+  }
+
+  /// Query interfaces of Interface
+  StatusCode queryInterface(const InterfaceID& riid, void** ppv) override;
+  AllenConfiguration(std::string name, ISvcLocator* svcloc);
+
+  ~AllenConfiguration();
+
+  Gaudi::Property<int> eps {this, "EventsPerSlice", 1000};
+  Gaudi::Property<bool> nonStop {this, "NonStop", true};
+  Gaudi::Property<bool> withMPI {this, "MPI", true};
+  Gaudi::Property<std::string> receivers {this, "Receivers", ""};
+  Gaudi::Property<bool> check {this, "CheckMC", false};
+  Gaudi::Property<unsigned int> nThreads {this, "NThreads", 8};
+  Gaudi::Property<unsigned int> nSlices {this, "NSlices", 16};
+  Gaudi::Property<std::string> output {this, "Output", ""};
+  Gaudi::Property<std::string> device {this, "Device", "0"};
+  Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json"};
+  Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
+  Gaudi::Property<std::vector<std::string>> input {this, "Input"};
+};
-- 
GitLab


From 156840ca81d94df45f8762b06faae09b0c3a1e12 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 26 May 2021 16:08:29 +0200
Subject: [PATCH 002/120] WIP move MEP provider from Allen to AllenOnline; use
 EventBuilding headers

---
 AllenOnline/CMakeLists.txt           |   23 +-
 AllenOnline/src/AllenApplication.cpp |   97 +-
 AllenOnline/src/AllenApplication.h   |   21 +-
 AllenOnline/src/AllenConfiguration.h |   15 +-
 AllenOnline/src/EBProvider.h         | 1293 ++++++++++++++++++++++++++
 AllenOnline/src/MEPProvider.cpp      | 1068 +++++++++++++++++++++
 AllenOnline/src/MEPProvider.h        |  263 ++++++
 AllenOnline/src/ReadMEP.cpp          |   97 ++
 AllenOnline/src/ReadMEP.h            |   16 +
 AllenOnline/src/TransposeMEP.cpp     |  388 ++++++++
 AllenOnline/src/TransposeMEP.h       |  137 +++
 AllenOnline/src/WriteMDF.h           |   25 +
 12 files changed, 3387 insertions(+), 56 deletions(-)
 create mode 100644 AllenOnline/src/EBProvider.h
 create mode 100644 AllenOnline/src/MEPProvider.cpp
 create mode 100644 AllenOnline/src/MEPProvider.h
 create mode 100644 AllenOnline/src/ReadMEP.cpp
 create mode 100644 AllenOnline/src/ReadMEP.h
 create mode 100644 AllenOnline/src/TransposeMEP.cpp
 create mode 100644 AllenOnline/src/TransposeMEP.h
 create mode 100644 AllenOnline/src/WriteMDF.h

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 345c2d35d..e474508a8 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -9,14 +9,14 @@ gaudi_depends_on_subdirs(Online/OnlineKernel
                          Online/RPC
                          Tools/ZeroMQ
                          Online/Parsers
+                         Online/EventBuilding
                          Dumpers/BinaryDumpers)
 
-include(AllenExport)
-
 find_package(cppgsl REQUIRED)
 find_package(ZMQ REQUIRED)
 find_package(Boost REQUIRED)
 find_package(ROOT REQUIRED)
+
 include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
   ${ZMQ_INCLUDE_DIRS}
   ${CPPGSL_INCLUDE_DIR}
@@ -30,8 +30,21 @@ include_directories(${CMAKE_SOURCE_DIR}/main/include)
 
 gaudi_add_module(AllenOnline
                  src/*.cpp
-                 INCLUDE_DIRS OnlineBase ROOT Online/DIM GaudiKernel cppgsl Online/OnlineKernel Online/ZeroMQ
-                 LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQ ZMQLib RPC Parsers
-                                OnlineKernel AllenLib DAQEventLib DAQKernelLib GaudiAlgLib
+                 INCLUDE_DIRS OnlineBase ROOT Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
+                 LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQLib RPC Parsers EventBuildingLib MDFLib
+                                OnlineKernel Allen::AllenLib Allen::BinaryDumpers DAQEventLib DAQKernelLib GaudiAlgLib
                                 PrKernel VPDetLib UTDetLib UTKernelLib
                                 FTDetLib -lrt)
+
+# MPI
+find_package(MPI QUIET COMPONENTS C CXX)
+if (MPI_FOUND)
+  message(STATUS "Found MPI ${MPI_VERSION}: ${MPI_CXX_COMPILER}")
+  #hwloc is also needed
+  find_package(PkgConfig REQUIRED)
+  pkg_check_modules(HWLOC REQUIRED IMPORTED_TARGET hwloc)
+
+  target_link_libraries(AllenOnline PUBLIC MPI::MPI_CXX PkgConfig::HWLOC open-pal)
+  target_include_directories(AllenOnline SYSTEM PUBLIC ${MPI_CXX_INCLUDE_DIRS})
+  target_compile_definitions(AllenOnline PRIVATE HAVE_MPI)
+endif()
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index a19c58f64..6d12baa93 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -5,6 +5,7 @@
 #include <thread>
 #include <cmath>
 #include <regex>
+#include <unordered_set>
 
 #include <GaudiKernel/IJobOptionsSvc.h>
 #include <GaudiKernel/IMessageSvc.h>
@@ -22,6 +23,8 @@
 
 #include <GaudiOnline/OnlineApplication.h>
 
+#include <Allen/BankTypes.h>
+#include <Allen/BankMapping.h>
 #include <Allen/Allen.h>
 
 #ifdef HAVE_MPI
@@ -30,6 +33,9 @@
 
 #include "AllenConfiguration.h"
 #include "AllenApplication.h"
+#include "MEPProvider.h"
+
+// #include "EBProvider.h"
 
 namespace {
   using namespace std::string_literals;
@@ -63,9 +69,9 @@ AllenApplication::AllenApplication(Options opts) : OnlineApplication(opts) {}
 // Default destructor
 AllenApplication::~AllenApplication()
 {
-  if (m_handle) {
-    dlclose(m_handle);
-  }
+  // if (m_handle) {
+  //   dlclose(m_handle);
+  // }
 }
 
 /// Stop the application                             (RUNNING    -> READY)
@@ -101,22 +107,22 @@ int AllenApplication::configureApplication()
   if (ret != Online::ONLINE_OK) return ret;
 
   // dlopen libAllenLib
-  m_handle = dlopen("libAllenLib.so", RTLD_LAZY);
-  if (!m_handle) {
-    m_logger->error("Failed to dlopen libAllenLib");
-    return Online::ONLINE_ERROR;
-  }
-
-  // reset errors
-  dlerror();
-  // load the symbol
-  m_allenFun = (allen_t) dlsym(m_handle, "allen");
-  const char* dlsym_error = dlerror();
-  if (dlsym_error) {
-    m_logger->error("Failed to get 'allen' from libAllenLib");
-    dlclose(m_handle);
-    return Online::ONLINE_ERROR;
-  }
+  // m_handle = dlopen("libAllenLib.so", RTLD_LAZY);
+  // if (!m_handle) {
+  //   m_logger->error("Failed to dlopen libAllenLib");
+  //   return Online::ONLINE_ERROR;
+  // }
+
+  // // reset errors
+  // dlerror();
+  // // load the symbol
+  // m_allenFun = (allen_t) dlsym(m_handle, "allen");
+  // const char* dlsym_error = dlerror();
+  // if (dlsym_error) {
+  //   m_logger->error("Failed to get 'allen' from libAllenLib");
+  //   dlclose(m_handle);
+  //   return Online::ONLINE_ERROR;
+  // }
 
   SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
 
@@ -167,6 +173,11 @@ int AllenApplication::configureApplication()
     }
   }
 
+  m_provider = makeProvider();
+  if (!m_provider) {
+    return Online::ONLINE_ERROR;
+  }
+
   m_allenControl = m_zmqSvc->socket(zmq::PAIR);
   m_allenControl->bind(m_controlConnection.c_str());
 
@@ -305,7 +316,6 @@ void AllenApplication::allenLoop()
   //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
   //--device 23:00.0
   std::map<std::string, std::string> allen_options = {{"events-per-slice", std::to_string(m_allenConfig->eps.value())},
-                                                      {"non-stop", std::to_string(m_allenConfig->nonStop.value())},
                                                       {"c", std::to_string(m_allenConfig->check.value())},
                                                       {"v", std::to_string(6 - m_config->outputLevel())},
                                                       {"t", std::to_string(m_allenConfig->nThreads.value())},
@@ -321,29 +331,36 @@ void AllenApplication::allenLoop()
     allen_options["s"] = std::to_string(m_allenConfig->nSlices.value());
   }
 
-  auto const& input = m_allenConfig->input.value();
-  if (m_allenConfig->withMPI.value() == true) {
-    if (!m_allenConfig->receivers.value().empty()) {
-      allen_options["with-mpi"] = m_allenConfig->receivers.value();
-    }
-    else {
-      allen_options["with-mpi"] = "1";
-    }
-  }
-  else if (input.empty()) {
+  if (!m_allenConfig->withMPI.value() && m_allenConfig->input.value().empty()) {
     m_logger->throwError("No input files specified");
   }
-  else {
-    std::stringstream ss;
-    bool mep = false;
-    for (size_t i = 0; i < input.size(); ++i) {
-      if (i != 0) ss << ",";
-      if (input[i].find(".mep") != std::string::npos) mep = true;
-      ss << input[i];
+
+  allen(allen_options, m_updater, m_provider.get(), m_zmqSvc.get(), m_controlConnection);
+}
+
+std::unique_ptr<IInputProvider> AllenApplication::makeProvider()
+{
+  MEPProviderConfig config {false,                // verify MEP checksums
+                            10,                   // number of read buffers
+                            m_allenConfig->transpose.value() ? 4u : 1u, // number of transpose threads
+                            4,                                          // MPI sliding window size
+                            m_allenConfig->withMPI.value(),             // Receive from MPI or read files
+                            m_allenConfig->nonStop.value(),             // Loop over input non-stop
+                            m_allenConfig->transpose.value(),           // MEPs should be transposed to Allen layout
+                            m_allenConfig->runChanges.value(),          // Whether to split slices by run number
+                            m_allenConfig->receivers.value()};          // Map of receiver to MPI rank to receive from
+
+  std::unordered_set<BankTypes> bankTypes;
+  for (auto bt : m_allenConfig->bankTypes.value()) {
+    auto it = Allen::bank_types.find(bt);
+    if (it == Allen::bank_types.end()) {
+      m_logger->error(std::string{"No Allen bank type defined for requested bank type "} + LHCb::RawBank::typeName(bt));
+      return {};
+    }
+    else {
+      bankTypes.insert(it->second);
     }
-    auto files = ss.str();
-    allen_options[(mep ? "mep" : "mdf")] = files;
   }
 
-  m_allenFun(allen_options, m_updater, m_zmqSvc.get(), m_controlConnection);
+  return std::make_unique<MEPProvider>(m_allenConfig->nSlices.value(), m_allenConfig->eps.value(), std::optional<size_t>{}, m_allenConfig->input.value(), bankTypes, config);
 }
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 907626573..83d6b0386 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -21,8 +21,6 @@
 #include <GaudiOnline/OnlineApplication.h>
 #include <ZeroMQ/IZeroMQSvc.h>
 
-#include <Allen/Allen.h>
-
 #include "AllenConfiguration.h"
 
 class AllenApplication : public Online::OnlineApplication {
@@ -65,6 +63,9 @@ public:
   bool initMPI();
 
 private:
+
+  std::unique_ptr<IInputProvider> makeProvider();
+
   /// Reference to the monitoring service
   SmartIF<IMonitorSvc> m_monSvc;
 
@@ -81,16 +82,18 @@ private:
 
   std::string m_controlConnection = "inproc://AllenApplicationControl";
 
+  std::unique_ptr<IInputProvider> m_provider;
+
   // dlopen stuff to workaround segfault in genconf.exe
-  void* m_handle = nullptr;
-  typedef int (
-    *allen_t)(std::map<std::string, std::string>, Allen::NonEventData::IUpdater*, IZeroMQSvc* zmqSvc, std::string_view);
-  allen_t m_allenFun = nullptr;
+  // void* m_handle = nullptr;
+  // typedef int (
+  //   *allen_t)(std::map<std::string, std::string>, Allen::NonEventData::IUpdater*, IZeroMQSvc* zmqSvc, std::string_view);
+  // allen_t m_allenFun = nullptr;
 
   std::thread m_allenThread;
   std::optional<zmq::socket_t> m_allenControl;
 
-  char** m_mpiArgv = nullptr;
-  int m_mpiArgc = 1;
-  int m_rank = -1;
+  // char** m_mpiArgv = nullptr;
+  // int m_mpiArgc = 1;
+  // int m_rank = -1;
 };
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index 4a7a431ce..0ec6a6922 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -1,6 +1,7 @@
 #pragma once
 
-#include "GaudiKernel/Service.h"
+#include <GaudiKernel/Service.h>
+#include <Dumpers/RawBankProperty.h>
 
 class AllenConfiguration : public Service {
 public:
@@ -20,14 +21,24 @@ public:
 
   Gaudi::Property<int> eps {this, "EventsPerSlice", 1000};
   Gaudi::Property<bool> nonStop {this, "NonStop", true};
+  Gaudi::Property<bool> transpose {this, "TransposeMEP", false};
   Gaudi::Property<bool> withMPI {this, "MPI", true};
-  Gaudi::Property<std::string> receivers {this, "Receivers", ""};
+  Gaudi::Property<std::map<std::string, int>> receivers {this, "Receivers"};
   Gaudi::Property<bool> check {this, "CheckMC", false};
   Gaudi::Property<unsigned int> nThreads {this, "NThreads", 8};
   Gaudi::Property<unsigned int> nSlices {this, "NSlices", 16};
   Gaudi::Property<std::string> output {this, "Output", ""};
+  Gaudi::Property<bool> runChanges {this, "EnableRunChanges", true};
   Gaudi::Property<std::string> device {this, "Device", "0"};
   Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json"};
   Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
   Gaudi::Property<std::vector<std::string>> input {this, "Input"};
+  Gaudi::Property<std::unordered_set<LHCb::RawBank::BankType>> bankTypes{this, "BankTypes",
+                                                                  {LHCb::RawBank::VP,
+                                                                   LHCb::RawBank::UT,
+                                                                   LHCb::RawBank::FTCluster,
+                                                                   LHCb::RawBank::Muon,
+                                                                   LHCb::RawBank::ODIN,
+                                                                   LHCb::RawBank::EcalPacked,
+                                                                   LHCb::RawBank::HcalPacked}};
 };
diff --git a/AllenOnline/src/EBProvider.h b/AllenOnline/src/EBProvider.h
new file mode 100644
index 000000000..43ef89f07
--- /dev/null
+++ b/AllenOnline/src/EBProvider.h
@@ -0,0 +1,1293 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#pragma once
+
+#include <thread>
+#include <vector>
+#include <array>
+#include <deque>
+#include <mutex>
+#include <atomic>
+#include <chrono>
+#include <algorithm>
+#include <numeric>
+#include <condition_variable>
+#include <cassert>
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+
+#include <InputProvider.h>
+#include <BankTypes.h>
+#include <Event/RawBank.h>
+#include <MEPTools.h>
+#include <SliceUtils.h>
+
+#ifdef HAVE_HWLOC
+#include <hwloc.h>
+#endif
+
+namespace {
+  using namespace Allen::Units;
+  using namespace std::string_literals;
+} // namespace
+
+/**
+ * @brief      Configuration parameters for the MEPProvider
+ */
+struct EBProviderConfig {
+  // check the MDF checksum if it is available
+  bool check_checksum = false;
+
+  // number of prefetch buffers
+  size_t n_buffers = 8;
+
+  // number of transpose threads
+  size_t n_transpose_threads = 5;
+
+  int window_size = 4;
+
+  // Use MPI and number of receivers
+  bool mpi = false;
+
+  bool non_stop = true;
+
+  bool transpose_mep = false;
+
+  bool split_by_run = false;
+
+  size_t n_receivers() const { return receivers.size(); }
+
+  // Mapping of receiver card to MPI rank to receive from
+  std::map<std::string, int> receivers;
+};
+
+/**
+ * @brief      Provide transposed events from MDF files
+ *
+ * @details    The provider has three main components
+ *             - a prefetch thread to read from the current input
+ *               file into prefetch buffers
+ *             - N transpose threads that read from prefetch buffers
+ *               and fill the per-bank-type slices with transposed sets
+ *               of banks and the offsets to individual bank inside a
+ *               given set
+ *             - functions to obtain a transposed slice and declare it
+ *               for refilling
+ *
+ *             Access to prefetch buffers and slices is synchronised
+ *             using mutexes and condition variables.
+ *
+ * @param      Number of slices to fill
+ * @param      Number of events per slice
+ * @param      MDF filenames
+ * @param      Configuration struct
+ *
+ */
+template<BankTypes... Banks>
+class EBProvider final : public InputProvider<MEPProvider<Banks...>> {
+public:
+  MEPProvider(
+    size_t n_slices,
+    size_t events_per_slice,
+    std::optional<size_t> n_events,
+    std::vector<std::string> connections,
+    std::unordered_set<BankTypes> const& bank_types,
+    MEPProviderConfig config = MEPProviderConfig {}) noexcept(false) :
+    InputProvider n_slices, events_per_slice, bank_types,
+      config.transpose_mep ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP,
+      n_events},
+    m_buffer_status(config.n_buffers), m_slice_free(n_slices, true), m_banks_count {0}, m_event_ids {n_slices},
+    m_connections {std::move(connections)}, m_config {config}
+  {
+
+    if (m_config.transpose_mep) {
+      info_cout << "Providing events in Allen layout by transposing MEPs\n";
+    }
+    else {
+      info_cout << "Providing events in MEP layout\n";
+    }
+
+    m_buffer_transpose = m_buffer_status.begin();
+    m_buffer_reading = m_buffer_status.begin();
+
+    if (m_config.mpi) {
+      init_mpi();
+    }
+    else {
+      m_read_buffers.resize(m_config.n_buffers);
+      m_net_slices.resize(m_config.n_buffers);
+    }
+
+    // Initialize the current input filename
+    m_current = m_connections.begin();
+
+    // Allocate space to store event ids
+    for (size_t n = 0; n < n_slices; ++n) {
+      m_event_ids[n].reserve(events_per_slice);
+    }
+
+    // Cached bank LHCb bank type to Allen bank type mapping
+    m_bank_ids = bank_ids();
+
+    // Reserve 1MB for decompression
+    m_compress_buffer.reserve(1u * MB);
+
+    // start MPI receive or MEP reading thread
+    if (m_config.mpi) {
+#ifdef HAVE_MPI
+      m_input_thread = std::thread {&MEPProvider::mpi_read, this};
+#endif
+    }
+    else {
+      m_input_thread = std::thread {&MEPProvider::mep_read, this};
+    }
+
+    // Sanity check on the number of buffers and threads
+    if (m_config.n_buffers <= 1) {
+      warning_cout << "too few read buffers requested, setting it to 2\n";
+      m_config.n_buffers = 2;
+    }
+
+    if (m_config.n_transpose_threads > m_config.n_buffers - 1) {
+      warning_cout << "too many transpose threads requested with respect "
+                      "to the number of read buffers; reducing the number of threads to "
+                   << m_config.n_buffers - 1;
+      m_config.n_transpose_threads = m_config.n_buffers - 1;
+    }
+
+    // Start the transpose threads
+    if (m_transpose_threads.empty() && !m_read_error) {
+      for (size_t i = 0; i < m_config.n_transpose_threads; ++i) {
+        m_transpose_threads.emplace_back([this, i] { transpose(i); });
+      }
+    }
+  }
+
+  /// Destructor
+  virtual ~MEPProvider() noexcept(false)
+  {
+
+    // Set flag to indicate the prefetch thread should exit, wake it
+    // up and join it
+    m_done = true;
+    m_transpose_done = true;
+    m_mpi_cond.notify_one();
+    m_control_cond.notify_all();
+    m_input_thread.join();
+
+    // Set a flat to indicate all transpose threads should exit, wake
+    // them up and join the threads. Ensure any waiting calls to
+    // get_slice also return.
+    m_mpi_cond.notify_all();
+    m_transpose_cond.notify_all();
+    m_slice_cond.notify_all();
+
+    for (auto& thread : m_transpose_threads) {
+      thread.join();
+    }
+
+#ifdef HAVE_MPI
+    for (auto* buf : m_mpi_buffers) {
+      cudaCheck(cudaHostUnregister(buf));
+      MPI_Free_mem(buf);
+    }
+#ifdef HAVE_HWLOC
+    if (m_config.mpi) {
+      hwloc_topology_destroy(m_topology);
+    }
+#endif
+#endif
+  }
+
+  /**
+   * @brief      Obtain event IDs of events stored in a given slice
+   *
+   * @param      slice index
+   *
+   * @return     EventIDs of events in given slice
+   */
+  EventIDs event_ids(size_t slice_index, std::optional<size_t> first = {}, std::optional<size_t> last = {})
+    const override
+  {
+    auto const& ids = m_event_ids[slice_index];
+    return {ids.begin() + (first ? *first : 0), ids.begin() + (last ? *last : ids.size())};
+  }
+
+  /**
+   * @brief      Obtain banks from a slice
+   *
+   * @param      BankType
+   * @param      slice index
+   *
+   * @return     Banks and their offsets
+   */
+  BanksAndOffsets banks(BankTypes bank_type, size_t slice_index) const override
+  {
+    auto ib = to_integral<BankTypes>(bank_type);
+    auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
+
+    BanksAndOffsets bno;
+    auto& spans = std::get<0>(bno);
+    spans.reserve(banks.size());
+    for (auto s : banks) {
+      spans.emplace_back(s);
+    }
+    std::get<1>(bno) = m_config.transpose_mep ? offsets[offsets_size - 1] : data_size;
+    std::get<2>(bno) = offsets;
+    return bno;
+  }
+
+  /**
+   * @brief      Get a slice that is ready for processing; thread-safe
+   *
+   * @param      optional timeout
+   *
+   * @return     (good slice, timed out, slice index, number of events in slice)
+   */
+  std::tuple<bool, bool, bool, size_t, size_t, uint> get_slice(
+    std::optional<unsigned int> timeout = {}) override
+  {
+    bool timed_out = false, done = false;
+    size_t slice_index = 0, n_filled = 0;
+    uint run_no = 0;
+    std::unique_lock<std::mutex> lock {m_transpose_mut};
+
+    if (!m_read_error) {
+      // If no transposed slices are ready for processing, wait until
+      // one is; use a timeout if requested
+      if (m_transposed.empty()) {
+        auto wakeup = [this] {
+          auto n_writable = count_writable();
+          return (
+            !m_transposed.empty() || m_read_error || (m_transpose_done && n_writable == m_buffer_status.size()) ||
+            (m_stopping && n_writable == m_buffer_status.size()));
+        };
+        if (timeout) {
+          timed_out = !m_transpose_cond.wait_for(lock, std::chrono::milliseconds {*timeout}, wakeup);
+        }
+        else {
+          m_transpose_cond.wait(lock, wakeup);
+        }
+      }
+      if (!m_read_error && !m_transposed.empty() && (!timeout || (timeout && !timed_out))) {
+        std::tie(slice_index, n_filled) = m_transposed.front();
+        m_transposed.pop_front();
+        if (n_filled > 0) {
+          run_no = std::get<0>(m_event_ids[slice_index].front());
+        }
+      }
+    }
+
+    // Check if I/O and transposition is done and return a slice index
+    auto n_writable = count_writable();
+    done = ((m_transpose_done && m_transposed.empty()) || m_stopping) && n_writable == m_buffer_status.size();
+
+    if (timed_out && logger::verbosity() >= logger::verbose) {
+      this->debug_output(
+        "get_slice timed out; error " + std::to_string(m_read_error) + " done " + std::to_string(done) + " n_filled " +
+        std::to_string(n_filled));
+    }
+    else if (!timed_out) {
+      this->debug_output(
+        "get_slice returning " + std::to_string(slice_index) + "; error " + std::to_string(m_read_error) + " done " +
+        std::to_string(done) + " n_filled " + std::to_string(n_filled));
+    }
+
+    return {!m_read_error, done, timed_out, slice_index, m_read_error ? 0 : n_filled, run_no};
+  }
+
+  /**
+   * @brief      Declare a slice free for reuse; thread-safe
+   *
+   * @param      slice index
+   *
+   * @return     void
+   */
+  void slice_free(size_t slice_index) override
+  {
+    // Check if a slice was acually in use before and if it was, only
+    // notify the transpose threads that a free slice is available
+    bool freed = false, set_writable = false;
+    int i_buffer = 0;
+    {
+      std::unique_lock<std::mutex> lock {m_slice_mut};
+      if (!m_slice_free[slice_index]) {
+        m_slice_free[slice_index] = true;
+        freed = true;
+
+        // Clear relation between slice and buffer
+        i_buffer = std::get<0>(m_slice_to_buffer[slice_index]);
+        auto& status = m_buffer_status[i_buffer];
+        m_slice_to_buffer[slice_index] = {-1, 0, 0};
+
+        // If MEPs are not transposed and the respective buffer is no
+        // longer in use, set it to writable
+        if (
+          status.work_counter == 0 &&
+          (std::find_if(m_slice_to_buffer.begin(), m_slice_to_buffer.end(), [i_buffer](const auto& entry) {
+             return std::get<0>(entry) == i_buffer;
+           }) == m_slice_to_buffer.end())) {
+          status.writable = true;
+          set_writable = true;
+        }
+      }
+    }
+    if (freed) {
+      this->debug_output("Freed slice " + std::to_string(slice_index));
+      m_slice_cond.notify_one();
+    }
+    if (set_writable) {
+      this->debug_output("Set buffer " + std::to_string(i_buffer) + " writable");
+      m_mpi_cond.notify_one();
+    }
+  }
+
+  void event_sizes(
+    size_t const slice_index,
+    gsl::span<unsigned int const> const selected_events,
+    std::vector<size_t>& sizes) const override
+  {
+    int i_buffer = 0;
+    size_t interval_start = 0, interval_end = 0;
+    std::tie(i_buffer, interval_start, interval_end) = m_slice_to_buffer[slice_index];
+    auto const& blocks = std::get<2>(m_net_slices[i_buffer]);
+    for (unsigned int i = 0; i < selected_events.size(); ++i) {
+      auto event = selected_events[i];
+      sizes[i] +=
+        std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto& entry) {
+          auto const& block_header = std::get<0>(entry);
+          return s + bank_header_size + block_header.sizes[interval_start + event];
+        });
+    }
+  }
+
+  void copy_banks(size_t const slice_index, unsigned int const event, gsl::span<char> buffer) const override
+  {
+    auto [i_buffer, interval_start, interval_end] = m_slice_to_buffer[slice_index];
+    const auto mep_event = interval_start + event;
+
+    auto const& [mep_header, mpi_slice, blocks, fragment_offsets, slice_size] = m_net_slices[i_buffer];
+
+    unsigned char prev_type = 0;
+    auto block_index = 0;
+    size_t offset = 0;
+
+    for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
+      auto const& [block_header, block_data] = blocks[i_block];
+      auto lhcb_type = block_header.types[0];
+
+      if (prev_type != lhcb_type) {
+        block_index = 0;
+        prev_type = lhcb_type;
+      }
+
+      // All banks are taken directly from the block data to be able
+      // to treat banks needed by Allen and banks not needed by Allen
+      // in the same way
+      auto const fragment_offset = fragment_offsets[i_block][mep_event];
+      auto fragment_size = block_header.sizes[mep_event];
+
+      assert((offset + fragment_size) < static_cast<size_t>(buffer.size()));
+      offset += add_raw_bank(
+        block_header.types[mep_event],
+        mep_header.versions[i_block],
+        mep_header.source_ids[i_block],
+        {block_data.data() + fragment_offset, fragment_size},
+        buffer.data() + offset);
+      ++block_index;
+    }
+  }
+
+  int start() override
+  {
+    if (!m_started) {
+      std::unique_lock<std::mutex> lock {m_control_mutex};
+      this->debug_output("Starting", 0);
+      m_started = true;
+      m_stopping = false;
+    }
+    m_control_cond.notify_one();
+    return true;
+  };
+
+  int stop() override
+  {
+    {
+      std::unique_lock<std::mutex> lock {m_control_mutex};
+      m_stopping = true;
+      m_started = false;
+    }
+    // Make sure all threads wait for start in case they were waiting
+    // for a buffer
+    m_mpi_cond.notify_all();
+
+    return true;
+  };
+
+private:
+  void init_mpi()
+  {
+#ifdef HAVE_MPI
+
+    auto const& receivers = m_config.receivers;
+    m_domains.reserve(receivers.size());
+
+#ifdef HAVE_HWLOC
+    // Allocate and initialize topology object.
+    hwloc_topology_init(&m_topology);
+
+    // discover everything, in particular I/O devices like
+    // InfiniBand cards
+#if HWLOC_API_VERSION >= 0x20000
+    hwloc_topology_set_io_types_filter(m_topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT);
+#else
+    hwloc_topology_set_flags(m_topology, HWLOC_TOPOLOGY_FLAG_WHOLE_SYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES);
+#endif
+    // Perform the topology detection.
+    hwloc_topology_load(m_topology);
+
+    hwloc_obj_t osdev = nullptr;
+
+    if (!receivers.empty()) {
+      // Find NUMA domain of receivers
+      while ((osdev = hwloc_get_next_osdev(m_topology, osdev))) {
+        // We're interested in InfiniBand cards
+        if (osdev->attr->osdev.type == HWLOC_OBJ_OSDEV_OPENFABRICS) {
+          auto parent = hwloc_get_non_io_ancestor_obj(m_topology, osdev);
+          auto it = receivers.find(osdev->name);
+          if (it != receivers.end()) {
+            m_domains.emplace_back(it->second, parent->os_index);
+            this->debug_output(
+              "Located receiver device "s + it->first + " in NUMA domain " + std::to_string(parent->os_index));
+          }
+        }
+      }
+      if (m_domains.size() != receivers.size()) {
+        throw StrException {"Failed to locate some receiver devices "};
+      }
+    }
+#else
+    if (!receivers.empty()) {
+      info_cout << "hwloc is not available, assuming NUMA domain 0 for all receivers.\n";
+      for (auto [rec, rank] : receivers) {
+        m_domains.emplace_back(rank, 0);
+      }
+#endif
+  }
+  else { throw StrException {"MPI requested, but no receivers specified"}; }
+
+#ifdef HAVE_HWLOC
+  // Get last node. There's always at least one.
+  [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
+  assert(static_cast<size_t>(n_numa) == m_domains.size());
+
+  std::vector<hwloc_obj_t> numa_objs(m_config.n_receivers());
+  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
+    int numa_node = std::get<1>(m_domains[receiver]);
+    numa_objs[receiver] = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
+  }
+#endif
+
+  std::vector<size_t> packing_factors(m_config.n_receivers());
+  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
+    auto const receiver_rank = std::get<0>(m_domains[receiver]);
+    MPI_Recv(
+      &packing_factors[receiver],
+      1,
+      MPI_SIZE_T,
+      receiver_rank,
+      MPI::message::packing_factor,
+      MPI_COMM_WORLD,
+      MPI_STATUS_IGNORE);
+  }
+
+  if (!std::all_of(packing_factors.begin(), packing_factors.end(), [v = packing_factors.back()](auto const p) {
+        return p == v;
+      })) {
+    throw StrException {"All packing factors must be the same"};
+  }
+  else {
+    m_packing_factor = packing_factors.back();
+  }
+
+  // Allocate as many net slices as configured, of expected size
+  // Packing factor can be done dynamically if needed
+  size_t n_bytes = std::lround(m_packing_factor * average_event_size * bank_size_fudge_factor * kB);
+  for (size_t i = 0; i < m_config.n_buffers; ++i) {
+    char* contents = nullptr;
+    MPI_Alloc_mem(n_bytes, MPI_INFO_NULL, &contents);
+
+    // Only bind explicitly if there are multiple receivers,
+    // otherwise assume a memory allocation policy is in effect
+#ifdef HAVE_HWLOC
+    if (m_domains.size() > 1) {
+      auto const& numa_obj = numa_objs[numa_node];
+      auto s = hwloc_set_area_membind(
+        m_topology, contents, n_bytes, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
+      if (s != 0) {
+        throw StrException {"Failed to bind memory to node "s + std::to_string(numa_obj->os_index) + " " +
+                            strerror(errno)};
+      }
+    }
+#endif
+    cudaCheck(cudaHostRegister(contents, n_bytes, cudaHostRegisterDefault));
+    m_net_slices.emplace_back(
+      EB::Header {},
+      gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
+      MEP::Blocks {},
+      MEP::SourceOffsets {},
+      n_bytes);
+    m_mpi_buffers.emplace_back(contents);
+  }
+#else
+    error_cout << "MPI requested, but Allen was not built with MPI support.\n";
+    throw std::runtime_error {"No MPI supoprt"};
+#endif
+}
+
+size_t
+count_writable() const
+{
+  return std::accumulate(m_buffer_status.begin(), m_buffer_status.end(), 0ul, [](size_t s, BufferStatus const& stat) {
+    return s + stat.writable;
+  });
+}
+
+void allocate_storage(size_t i_read)
+{
+  if (m_sizes_known) return;
+
+  // Count number of banks per flavour
+  bool count_success = false;
+
+  // Offsets are to the start of the event, which includes the header
+  auto& [mep_header, mpi_slice, blocks, input_offsets, slice_size] = m_net_slices[i_read];
+  size_t n_blocks = mep_header.n_blocks;
+  // gsl::span<char const> block_span{mpi_slice.data() + mep_header.header_size(mep_header.n_blocks),
+  // mep_header.mep_size};
+  if (m_packing_factor == 0) {
+    m_packing_factor = mep_header.packing_factor;
+  }
+  else {
+    assert(mep_header.packing_factor == m_packing_factor);
+  }
+
+  auto eps = this->events_per_slice();
+  auto n_interval = m_packing_factor / eps;
+  auto rest = m_packing_factor % eps;
+  for (auto& s : m_buffer_status) {
+    s.intervals.reserve(2 * (n_interval + rest));
+  }
+
+  for (auto& [mep_header, mpi_slice, blocks, input_offsets, slice_size] : m_net_slices) {
+    // The number of blocks in a MEP is known, use it to allocate
+    // temporary storage used during transposition
+    blocks.resize(n_blocks);
+    input_offsets.resize(n_blocks);
+    for (auto& offsets : input_offsets) {
+      // info_cout << "Packing factor: " << mep_header.packing_factor << "\n";
+      offsets.resize(m_packing_factor + 1);
+    }
+  }
+
+  std::tie(count_success, m_banks_count) = MEP::fill_counts(mep_header, mpi_slice);
+
+  // Allocate slice memory that will contain transposed banks ready
+  // for processing by the Allen kernels
+  auto size_fun = [this, eps](BankTypes bank_type) -> std::tuple<size_t, size_t> {
+    auto it = BankSizes.find(bank_type);
+    auto ib = to_integral<BankTypes>(bank_type);
+    if (it == end(BankSizes)) {
+      throw std::out_of_range {std::string {"Bank type "} + std::to_string(ib) + " has no known size"};
+    }
+    // In case of direct MEP output, no memory should be allocated.
+    if (!m_config.transpose_mep) {
+      auto it = std::find(m_bank_ids.begin(), m_bank_ids.end(), to_integral(bank_type));
+      auto lhcb_type = std::distance(m_bank_ids.begin(), it);
+      auto n_blocks = m_banks_count[lhcb_type];
+      // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
+      return {0, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
+    }
+    else {
+      auto aps = eps < 100 ? 100 : eps;
+      return {std::lround(it->second * aps * bank_size_fudge_factor * kB), eps};
+    }
+  };
+  m_slices = allocate_slices<Banks...>(this->n_slices(), size_fun);
+  m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(this->n_slices(), std::make_tuple(-1, 0ul, 0ul));
+
+  if (!count_success) {
+    error_cout << "Failed to determine bank counts\n";
+    m_read_error = true;
+  }
+  else {
+    m_sizes_known = true;
+  }
+}
+
+/**
+ * @brief      Open an input file; called from the prefetch thread
+ *
+ * @return     success
+ */
+bool open_file() const
+{
+  bool good = false;
+
+  // Check if there are still files available
+  while (!good) {
+    // If looping on input is configured, do it
+    if (m_current == m_connections.end()) {
+      if (m_config.non_stop) {
+        m_current = m_connections.begin();
+      }
+      else {
+        break;
+      }
+    }
+
+    if (m_input) m_input->close();
+
+    m_input = MDF::open(*m_current, O_RDONLY);
+    if (m_input->good) {
+      info_cout << "Opened " << *m_current << "\n";
+      good = true;
+    }
+    else {
+      error_cout << "Failed to open " << *m_current << " " << strerror(errno) << "\n";
+      m_read_error = true;
+      return false;
+    }
+    ++m_current;
+  }
+  return good;
+}
+
+std::tuple<std::vector<BufferStatus>::iterator, size_t> get_mep_buffer(
+  std::function<bool(BufferStatus const&)> pred,
+  std::vector<BufferStatus>::iterator start,
+  std::unique_lock<std::mutex>& lock)
+{
+  // Obtain a prefetch buffer to read into, if none is available,
+  // wait until one of the transpose threads is done with its
+  // prefetch buffer
+  auto find_buffer = [this, start, &pred] {
+    auto it = std::find_if(start, m_buffer_status.end(), pred);
+    if (it == m_buffer_status.end()) {
+      it = std::find_if(m_buffer_status.begin(), start, pred);
+      if (it == start) it = m_buffer_status.end();
+    }
+    return it;
+  };
+
+  auto it = find_buffer();
+  if (it == m_buffer_status.end() && !m_transpose_done) {
+    m_mpi_cond.wait(lock, [this, &it, &find_buffer] {
+      it = find_buffer();
+      return it != m_buffer_status.end() || m_transpose_done || m_stopping;
+    });
+  }
+  return {it, distance(m_buffer_status.begin(), it)};
+}
+
+void set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events)
+{
+  if (n_events == 0) return;
+  const auto eps = this->events_per_slice();
+  auto n_interval = n_events / eps;
+  auto rest = n_events % eps;
+  if (rest) {
+    debug_cout << "Set interval (rest): " << n_interval * eps << "," << n_interval * eps + rest << "\n";
+    intervals.emplace_back(n_interval * eps, n_interval * eps + rest);
+  }
+  for (size_t i = n_interval; i != 0; --i) {
+    debug_cout << "Set interval: " << (i - 1) * eps << "," << i * eps << "\n";
+    intervals.emplace_back((i - 1) * eps, i * eps);
+  }
+}
+
+// mep reader thread
+void mep_read()
+{
+  bool receive_done = false;
+  EB::Header mep_header;
+
+  auto to_read = this->n_events();
+  if (to_read) debug_cout << "Reading " << *to_read << " events\n";
+  auto to_publish = 0;
+
+  while (!receive_done) {
+    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
+
+    // If we've been stopped, wait for start or exit
+    if (!m_started || m_stopping) {
+      std::unique_lock<std::mutex> lock {m_control_mutex};
+      this->debug_output("Waiting for start", 0);
+      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+    }
+
+    if (m_done) break;
+
+    // open the first file
+    if (!m_input && !open_file()) {
+      m_read_error = true;
+      m_mpi_cond.notify_one();
+      return;
+    }
+    size_t i_buffer;
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::tie(m_buffer_reading, i_buffer) =
+        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+      if (m_buffer_reading != m_buffer_status.end()) {
+        m_buffer_reading->writable = false;
+        assert(m_buffer_reading->work_counter == 0);
+      }
+      else {
+        continue;
+      }
+    }
+    if (m_done) {
+      receive_done = true;
+      break;
+    }
+
+    this->debug_output("Writing to MEP slice index " + std::to_string(i_buffer));
+
+    auto& read_buffer = m_read_buffers[i_buffer];
+    auto& [mep_header, buffer_span, blocks, input_offsets, buffer_size] = m_net_slices[i_buffer];
+
+    bool success = false, eof = false;
+
+    while (!success || eof) {
+      std::tie(eof, success, mep_header, buffer_span) = MEP::read_mep(*m_input, read_buffer);
+
+      if (!eof) {
+        debug_cout << "Read mep with packing factor " << mep_header.packing_factor << "\n";
+        if (to_read && success) {
+          to_publish = std::min(*to_read, size_t {mep_header.packing_factor});
+          *to_read -= to_publish;
+        }
+        else {
+          to_publish = mep_header.packing_factor;
+        }
+      }
+
+      if (!success) {
+        // Error encountered
+        m_read_error = true;
+        break;
+      }
+      else if ((to_read && *to_read == 0) || (eof && !open_file())) {
+        // Try to open the next file, if there is none, prefetching
+        // is done.
+        if (!m_read_error) {
+          this->debug_output("Prefetch done: eof and no more files");
+        }
+        receive_done = true;
+        break;
+      }
+    }
+
+    if (!m_sizes_known) {
+      allocate_storage(i_buffer);
+    }
+
+    // Notify a transpose thread that a new buffer of events is
+    // ready. If prefetching is done, wake up all threads
+    if (success) {
+      {
+        std::unique_lock<std::mutex> lock {m_mpi_mutex};
+
+        auto& status = m_buffer_status[i_buffer];
+        assert(status.work_counter == 0);
+
+        if (!eof && to_publish != 0) {
+          set_intervals(status.intervals, to_read ? to_publish : size_t {mep_header.packing_factor});
+        }
+        else {
+          // We didn't read anything, so free the buffer we got again
+          status.writable = true;
+        }
+      }
+      if (receive_done) {
+        m_done = receive_done;
+        this->debug_output("Prefetch notifying all");
+        m_mpi_cond.notify_all();
+      }
+      else if (!eof) {
+        this->debug_output("Prefetch notifying one");
+        m_mpi_cond.notify_one();
+      }
+    }
+    m_mpi_cond.notify_one();
+  }
+}
+
+#ifdef HAVE_MPI
+// MPI reader thread
+void mpi_read()
+{
+
+  int window_size = m_config.window_size;
+  std::vector<MPI_Request> requests(window_size);
+
+  // Iterate over the slices
+  size_t reporting_period = 5;
+  std::vector<std::tuple<size_t, size_t>> data_received(m_config.n_receivers());
+  std::vector<size_t> n_meps(m_config.n_receivers());
+  Timer t;
+  Timer t_origin;
+  bool error = false;
+
+  for (size_t i = 0; i < m_config.n_receivers(); ++i) {
+    auto [mpi_rank, numa_domain] = m_domains[i];
+    MPI_Recv(&n_meps[i], 1, MPI_SIZE_T, mpi_rank, MPI::message::number_of_meps, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
+  }
+  size_t number_of_meps = std::accumulate(n_meps.begin(), n_meps.end(), 0u);
+
+  size_t current_mep = 0;
+  while (!m_done && (m_config.non_stop || current_mep < number_of_meps)) {
+    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
+
+    // If we've been stopped, wait for start or exit
+    if (!m_started || m_stopping) {
+      std::unique_lock<std::mutex> lock {m_control_mutex};
+      this->debug_output("Waiting for start", 0);
+      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+    }
+
+    if (m_done) break;
+
+    // Obtain a prefetch buffer to read into, if none is available,
+    // wait until one of the transpose threads is done with its
+    // prefetch buffer
+    size_t i_buffer;
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::tie(m_buffer_reading, i_buffer) =
+        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+      if (m_buffer_reading != m_buffer_status.end()) {
+        m_buffer_reading->writable = false;
+        assert(m_buffer_reading->work_counter == 0);
+      }
+      else {
+        continue;
+      }
+    }
+
+    auto receiver = i_buffer % m_config.n_receivers();
+    auto [sender_rank, numa_node] = m_domains[receiver];
+
+    this->debug_output(
+      "Receiving from rank " + std::to_string(sender_rank) + " into buffer " + std::to_string(i_buffer) +
+      "  NUMA domain " + std::to_string(numa_node));
+
+    auto& [mep_header, buffer_span, blocks, input_offsets, buffer_size] = m_net_slices[i_buffer];
+    char*& contents = m_mpi_buffers[i_buffer];
+
+    size_t mep_size = 0;
+    MPI_Recv(&mep_size, 1, MPI_SIZE_T, sender_rank, MPI::message::event_size, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
+
+    // Reallocate if needed
+    if (mep_size > buffer_size) {
+      buffer_size = mep_size * bank_size_fudge_factor;
+      // Unregister memory
+      cudaCheck(cudaHostUnregister(contents));
+
+      // Free memory
+      MPI_Free_mem(contents);
+
+      // Allocate new memory
+      MPI_Alloc_mem(buffer_size, MPI_INFO_NULL, &contents);
+
+      // Only bind explicitly if there are multiple receivers,
+      // otherwise assume a memory allocation policy is in effect
+#ifdef HAVE_HWLOC
+      if (m_domains.size() > 1) {
+        // Bind memory to numa domain of receiving card
+        auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
+        auto s = hwloc_set_area_membind(
+          m_topology, contents, buffer_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
+        if (s != 0) {
+          m_read_error = true;
+          error_cout << "Failed to bind memory to node " << std::to_string(numa_node) << " " << strerror(errno) << "\n";
+          break;
+        }
+      }
+#endif
+      // Register memory with CUDA
+      try {
+        cudaCheck(cudaHostRegister(contents, buffer_size, cudaHostRegisterDefault));
+      } catch (std::invalid_argument const&) {
+        m_read_error = true;
+        break;
+      }
+
+      buffer_span = gsl::span {contents, static_cast<events_size>(buffer_size)};
+    }
+
+    // Number of full-size (MPI::mdf_chunk_size) messages
+    int n_messages = mep_size / MPI::mdf_chunk_size;
+    // Size of the last message (if the MFP size is not a multiple of MPI::mdf_chunk_size)
+    int rest = mep_size - n_messages * MPI::mdf_chunk_size;
+    // Number of parallel sends
+    int n_sends = n_messages > window_size ? window_size : n_messages;
+
+    // info_cout << MPI::rank_str() << "n_messages " << n_messages << ", rest " << rest << ", n_sends " << n_sends <<
+    // "\n";
+
+    // Initial parallel sends
+    for (int k = 0; k < n_sends; k++) {
+      char* message = contents + k * MPI::mdf_chunk_size;
+      MPI_Irecv(
+        message,
+        MPI::mdf_chunk_size,
+        MPI_BYTE,
+        sender_rank,
+        MPI::message::event_send_tag_start + k,
+        MPI_COMM_WORLD,
+        &requests[k]);
+    }
+    // Sliding window sends
+    for (int k = n_sends; k < n_messages; k++) {
+      int r;
+      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      char* message = contents + k * MPI::mdf_chunk_size;
+      MPI_Irecv(
+        message,
+        MPI::mdf_chunk_size,
+        MPI_BYTE,
+        sender_rank,
+        MPI::message::event_send_tag_start + k,
+        MPI_COMM_WORLD,
+        &requests[r]);
+    }
+    // Last send (if necessary)
+    if (rest) {
+      int r;
+      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      char* message = contents + n_messages * MPI::mdf_chunk_size;
+      MPI_Irecv(
+        message,
+        rest,
+        MPI_BYTE,
+        sender_rank,
+        MPI::message::event_send_tag_start + n_messages,
+        MPI_COMM_WORLD,
+        &requests[r]);
+    }
+    // Wait until all chunks have been sent
+    MPI_Waitall(n_sends, requests.data(), MPI_STATUSES_IGNORE);
+
+    mep_header = EB::Header {contents};
+    buffer_span = gsl::span {contents, static_cast<events_size>(mep_size)};
+
+    if (!m_sizes_known) {
+      allocate_storage(i_buffer);
+    }
+
+    auto& [meps_received, bytes_received] = data_received[receiver];
+    bytes_received += mep_size;
+    meps_received += 1;
+    if (t.get_elapsed_time() >= reporting_period) {
+      const auto seconds = t.get_elapsed_time();
+      auto total_rate = 0.;
+      auto total_bandwidth = 0.;
+      for (size_t i_rec = 0; i_rec < m_config.n_receivers(); ++i_rec) {
+        auto& [mr, br] = data_received[i_rec];
+        auto [rec_rank, rec_node] = m_domains[i_rec];
+
+        const double rate = (double) mr / seconds;
+        const double bandwidth = ((double) (br * 8)) / (1024 * 1024 * 1024 * seconds);
+        total_rate += rate;
+        total_bandwidth += bandwidth;
+        printf(
+          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s; Domain %2i; Rank %2i\n",
+          t_origin.get_elapsed_time(),
+          seconds,
+          rate,
+          bandwidth,
+          rec_node,
+          rec_rank);
+
+        br = 0;
+        mr = 0;
+      }
+      if (m_config.n_receivers() > 1) {
+        printf(
+          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s\n",
+          t_origin.get_elapsed_time(),
+          seconds,
+          total_rate,
+          total_bandwidth);
+      }
+      t.restart();
+    }
+
+    // Notify a transpose thread that a new buffer of events is
+    // ready. If prefetching is done, wake up all threads
+    if (!error) {
+      {
+        std::unique_lock<std::mutex> lock {m_mpi_mutex};
+        set_intervals(m_buffer_status[i_buffer].intervals, size_t {mep_header.packing_factor});
+        assert(m_buffer_status[i_buffer].work_counter == 0);
+      }
+      this->debug_output("Prefetch notifying one");
+      m_mpi_cond.notify_one();
+    }
+    m_mpi_cond.notify_one();
+
+    current_mep++;
+  }
+
+  if (!m_done) {
+    m_done = true;
+    this->debug_output("Prefetch notifying all");
+    m_mpi_cond.notify_all();
+  }
+}
+#endif
+
+/**
+ * @brief      Function to run in each thread transposing events
+ *
+ * @param      thread ID
+ *
+ * @return     void
+ */
+void transpose(int thread_id)
+{
+
+  size_t i_buffer = 0;
+  std::tuple<size_t, size_t> interval;
+  std::optional<size_t> slice_index;
+
+  bool good = false, transpose_full = false;
+  size_t n_transposed = 0;
+
+  auto has_intervals = [](BufferStatus const& s) { return !s.intervals.empty(); };
+
+  while (!m_read_error && !m_transpose_done) {
+    // Get a buffer to read from
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::tie(m_buffer_transpose, i_buffer) = get_mep_buffer(has_intervals, m_buffer_transpose, lock);
+      if (m_transpose_done) {
+        break;
+      }
+      else if (m_buffer_transpose == m_buffer_status.end()) {
+        continue;
+      }
+      auto& status = *m_buffer_transpose;
+      assert(!status.intervals.empty());
+
+      interval = status.intervals.back();
+      status.intervals.pop_back();
+
+      ++(status.work_counter);
+      status.writable = false;
+
+      this->debug_output(
+        "Got MEP slice index " + std::to_string(i_buffer) + " interval [" + std::to_string(std::get<0>(interval)) +
+          "," + std::to_string(std::get<1>(interval)) + ")",
+        thread_id);
+    }
+
+    // Get a slice to write to
+    if (!slice_index) {
+      this->debug_output("Getting slice index", thread_id);
+      auto it = m_slice_free.end();
+      {
+        std::unique_lock<std::mutex> lock {m_slice_mut};
+        it = find(m_slice_free.begin(), m_slice_free.end(), true);
+        if (it == m_slice_free.end()) {
+          this->debug_output("Waiting for free slice", thread_id);
+          m_slice_cond.wait(lock, [this, &it] {
+            it = std::find(m_slice_free.begin(), m_slice_free.end(), true);
+            return it != m_slice_free.end() || m_transpose_done;
+          });
+          // If transpose is done and there is no slice, we were
+          // woken up be the desctructor before a slice was declared
+          // free. In that case, exit without transposing
+          if (m_transpose_done && it == m_slice_free.end()) {
+            break;
+          }
+        }
+        *it = false;
+        slice_index = distance(m_slice_free.begin(), it);
+        this->debug_output("Got slice index " + std::to_string(*slice_index), thread_id);
+
+        // Keep track of what buffer this slice belonged to
+        m_slice_to_buffer[*slice_index] = {i_buffer, std::get<0>(interval), std::get<1>(interval)};
+      }
+    }
+
+    // Reset the slice
+    auto& event_ids = m_event_ids[*slice_index];
+    reset_slice<Banks...>(m_slices, *slice_index, event_ids, !m_config.transpose_mep);
+
+    // MEP data
+    auto& [mep_header, mep_data, blocks, source_offsets, slice_size] = m_net_slices[i_buffer];
+
+    // Fill blocks
+    MEP::find_blocks(mep_header, mep_data, blocks);
+
+    // Fill fragment offsets
+    MEP::fragment_offsets(blocks, source_offsets);
+
+    // Transpose or calculate offsets
+    if (m_config.transpose_mep) {
+      // Transpose the events into the slice
+      std::tie(good, transpose_full, n_transposed) = MEP::transpose_events(
+        m_slices,
+        *slice_index,
+        m_bank_ids,
+        this->types(),
+        m_banks_count,
+        event_ids,
+        mep_header,
+        blocks,
+        source_offsets,
+        interval);
+      this->debug_output(
+        "Transposed slice " + std::to_string(*slice_index) + "; good: " + std::to_string(good) +
+          "; full: " + std::to_string(transpose_full) + "; n_transposed:  " + std::to_string(n_transposed),
+        thread_id);
+    }
+    else {
+      // Calculate fragment offsets in MEP per sub-detector
+      std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
+        m_slices, *slice_index, m_bank_ids, this->types(), m_banks_count, event_ids, mep_header, blocks, interval);
+      this->debug_output("Calculated MEP offsets for slice " + std::to_string(*slice_index), thread_id);
+    }
+
+    if (m_read_error || !good) {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      auto& status = m_buffer_status[i_buffer];
+      --status.work_counter;
+      m_read_error = true;
+      m_transpose_cond.notify_one();
+      break;
+    }
+
+    // Notify any threads waiting in get_slice that a slice is available
+    {
+      std::unique_lock<std::mutex> lock {m_transpose_mut};
+      m_transposed.emplace_back(*slice_index, n_transposed);
+    }
+    m_transpose_cond.notify_one();
+    slice_index.reset();
+
+    // Check if the read buffer is now empty. If it is, it can be
+    // reused, otherwise give it to another transpose thread once a
+    // new target slice is available
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+
+      auto& status = m_buffer_status[i_buffer];
+      --status.work_counter;
+
+      if (n_transposed != std::get<1>(interval) - std::get<0>(interval)) {
+        status.intervals.emplace_back(std::get<0>(interval) + n_transposed, std::get<1>(interval));
+      }
+      else if (status.work_counter == 0) {
+        m_transpose_done =
+          m_done && std::all_of(m_buffer_status.begin(), m_buffer_status.end(), [](BufferStatus const& stat) {
+            return stat.intervals.empty() && stat.work_counter == 0;
+          });
+      }
+    }
+  }
+}
+
+// Slices
+size_t m_packing_factor = 0;
+std::vector<std::vector<char>> m_read_buffers;
+std::vector<char*> m_mpi_buffers;
+MEP::Slices m_net_slices;
+
+// data members for mpi thread
+bool m_started = false;
+bool m_stopping = false;
+std::mutex m_control_mutex;
+std::condition_variable m_control_cond;
+
+// data members for mpi thread
+std::mutex m_mpi_mutex;
+std::condition_variable m_mpi_cond;
+
+#ifdef HAVE_MPI
+std::vector<std::tuple<int, int>> m_domains;
+#endif
+
+#ifdef HAVE_HWLOC
+hwloc_topology_t m_topology;
+#endif
+
+std::vector<BufferStatus> m_buffer_status;
+std::vector<BufferStatus>::iterator m_buffer_transpose;
+std::vector<BufferStatus>::iterator m_buffer_reading;
+std::thread m_input_thread;
+
+// Atomics to flag errors and completion
+std::atomic<bool> m_done = false;
+mutable std::atomic<bool> m_read_error = false;
+std::atomic<bool> m_transpose_done = false;
+
+// Buffer to store data read from file if banks are compressed. The
+// decompressed data will be written to the buffers
+mutable std::vector<char> m_compress_buffer;
+
+// Storage to read the header into for each event
+mutable LHCb::MDFHeader m_header;
+
+// Allen IDs of LHCb raw banks
+std::vector<int> m_bank_ids;
+
+// Memory slices, N for each raw bank type
+Slices m_slices;
+std::vector<std::tuple<int, size_t, size_t>> m_slice_to_buffer;
+
+// Mutex, condition varaible and queue for parallel transposition of slices
+std::mutex m_transpose_mut;
+std::condition_variable m_transpose_cond;
+std::deque<std::tuple<size_t, size_t>> m_transposed;
+
+// Keep track of what slices are free
+std::mutex m_slice_mut;
+std::condition_variable m_slice_cond;
+std::vector<bool> m_slice_free;
+
+// Threads transposing data
+std::vector<std::thread> m_transpose_threads;
+
+// Array to store the number of banks per bank type
+mutable std::array<unsigned int, LHCb::NBankTypes> m_banks_count;
+mutable bool m_sizes_known = false;
+
+// Run and event numbers present in each slice
+std::vector<EventIDs> m_event_ids;
+
+// File names to read
+std::vector<std::string> m_connections;
+
+// Storage for the currently open input file
+mutable std::optional<Allen::IO> m_input;
+
+// Iterator that points to the filename of the currently open file
+mutable std::vector<std::string>::const_iterator m_current;
+
+// Input data loop counter
+mutable size_t m_loop = 0;
+
+// Configuration struct
+MEPProviderConfig m_config;
+
+using base_class = InputProvider<MEPProvider<Banks...>>;
+}
+;
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
new file mode 100644
index 000000000..9b8e0f07c
--- /dev/null
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -0,0 +1,1068 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+
+#include <Allen/Logger.h>
+#include <Allen/InputProvider.h>
+#include <Allen/BankTypes.h>
+#include <Allen/Timer.h>
+#include <Allen/SliceUtils.h>
+
+#include <MDF/StreamDescriptor.h>
+#include <MDF/MDFHeader.h>
+
+#include <Event/RawBank.h>
+
+#include "TransposeMEP.h"
+#include "WriteMDF.h"
+#include "ReadMEP.h"
+
+#ifdef HAVE_MPI
+#include "MPIConfig.h"
+#include <hwloc.h>
+#endif
+
+#include "MEPProvider.h"
+
+
+MEPProvider::MEPProvider(
+  size_t n_slices,
+  size_t events_per_slice,
+  std::optional<size_t> n_events,
+  std::vector<std::string> connections,
+  std::unordered_set<BankTypes> const& bank_types,
+  MEPProviderConfig config) noexcept(false) :
+InputProvider {n_slices, events_per_slice, bank_types,
+  config.transpose_mep ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP,
+  n_events},
+  m_buffer_status(config.n_buffers), m_slice_free(n_slices, true), m_banks_count {0}, m_event_ids {n_slices},
+  m_connections {std::move(connections)}, m_config {config}
+{
+
+  if (m_config.transpose_mep) {
+    info_cout << "Providing events in Allen layout by transposing MEPs\n";
+  }
+  else {
+    info_cout << "Providing events in MEP layout\n";
+  }
+
+  m_buffer_transpose = m_buffer_status.begin();
+  m_buffer_reading = m_buffer_status.begin();
+
+  if (m_config.mpi) {
+    init_mpi();
+  }
+  else {
+    m_read_buffers.resize(m_config.n_buffers);
+    m_net_slices.resize(m_config.n_buffers);
+  }
+
+  // Initialize the current input filename
+  m_current = m_connections.begin();
+
+  // Allocate space to store event ids
+  for (size_t n = 0; n < n_slices; ++n) {
+    m_event_ids[n].reserve(events_per_slice);
+  }
+
+  // Cached bank LHCb bank type to Allen bank type mapping
+  m_bank_ids = bank_ids();
+
+  // Reserve 1MB for decompression
+  m_compress_buffer.reserve(1u * MB);
+
+  // start MPI receive or MEP reading thread
+  if (m_config.mpi) {
+  #ifdef HAVE_MPI
+    m_input_thread = std::thread {&MEPProvider::mpi_read, this};
+  #else
+    throw StrException {"MPI requested, but no MPI support built in."};
+  #endif
+  }
+  else {
+    m_input_thread = std::thread {&MEPProvider::mep_read, this};
+  }
+
+  // Sanity check on the number of buffers and threads
+  if (m_config.n_buffers <= 1) {
+    warning_cout << "too few read buffers requested, setting it to 2\n";
+    m_config.n_buffers = 2;
+  }
+
+  if (m_config.n_transpose_threads > m_config.n_buffers - 1) {
+    warning_cout << "too many transpose threads requested with respect "
+                    "to the number of read buffers; reducing the number of threads to "
+                 << m_config.n_buffers - 1;
+    m_config.n_transpose_threads = m_config.n_buffers - 1;
+  }
+
+  // Start the transpose threads
+  if (m_transpose_threads.empty() && !m_read_error) {
+    for (size_t i = 0; i < m_config.n_transpose_threads; ++i) {
+      m_transpose_threads.emplace_back([this, i] { transpose(i); });
+    }
+  }
+}
+
+EventIDs MEPProvider::event_ids(size_t slice_index, std::optional<size_t> first, std::optional<size_t> last) const
+{
+  auto const& ids = m_event_ids[slice_index];
+  return {ids.begin() + (first ? *first : 0), ids.begin() + (last ? *last : ids.size())};
+}
+
+BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) const
+{
+  auto ib = to_integral<BankTypes>(bank_type);
+  auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
+
+  BanksAndOffsets bno;
+  auto& spans = std::get<0>(bno);
+  spans.reserve(banks.size());
+  for (auto s : banks) {
+    spans.emplace_back(s);
+  }
+  std::get<1>(bno) = m_config.transpose_mep ? offsets[offsets_size - 1] : data_size;
+  std::get<2>(bno) = offsets;
+  return bno;
+}
+
+std::tuple<bool, bool, bool, size_t, size_t, uint>
+MEPProvider::get_slice(std::optional<unsigned int> timeout)
+{
+  bool timed_out = false, done = false;
+  size_t slice_index = 0, n_filled = 0;
+  uint run_no = 0;
+  std::unique_lock<std::mutex> lock {m_transpose_mut};
+
+  if (!m_read_error) {
+    // If no transposed slices are ready for processing, wait until
+    // one is; use a timeout if requested
+    if (m_transposed.empty()) {
+      auto wakeup = [this] {
+        auto n_writable = count_writable();
+        return (
+          !m_transposed.empty() || m_read_error || (m_transpose_done && n_writable == m_buffer_status.size()) ||
+          (m_stopping && n_writable == m_buffer_status.size()));
+      };
+      if (timeout) {
+        timed_out = !m_transpose_cond.wait_for(lock, std::chrono::milliseconds {*timeout}, wakeup);
+      }
+      else {
+        m_transpose_cond.wait(lock, wakeup);
+      }
+    }
+    if (!m_read_error && !m_transposed.empty() && (!timeout || (timeout && !timed_out))) {
+      std::tie(slice_index, n_filled) = m_transposed.front();
+      m_transposed.pop_front();
+      if (n_filled > 0) {
+        run_no = std::get<0>(m_event_ids[slice_index].front());
+      }
+    }
+  }
+
+  // Check if I/O and transposition is done and return a slice index
+  auto n_writable = count_writable();
+  done = ((m_transpose_done && m_transposed.empty()) || m_stopping) && n_writable == m_buffer_status.size();
+
+  if (timed_out && logger::verbosity() >= logger::verbose) {
+    this->debug_output(
+      "get_slice timed out; error " + std::to_string(m_read_error) + " done " + std::to_string(done) + " n_filled " +
+      std::to_string(n_filled));
+  }
+  else if (!timed_out) {
+    this->debug_output(
+      "get_slice returning " + std::to_string(slice_index) + "; error " + std::to_string(m_read_error) + " done " +
+      std::to_string(done) + " n_filled " + std::to_string(n_filled));
+  }
+
+  return {!m_read_error, done, timed_out, slice_index, m_read_error ? 0 : n_filled, run_no};
+}
+
+void MEPProvider::slice_free(size_t slice_index)
+{
+  // Check if a slice was acually in use before and if it was, only
+  // notify the transpose threads that a free slice is available
+  bool freed = false, set_writable = false;
+  int i_buffer = 0;
+  {
+    std::unique_lock<std::mutex> lock {m_slice_mut};
+    if (!m_slice_free[slice_index]) {
+      m_slice_free[slice_index] = true;
+      freed = true;
+
+      // Clear relation between slice and buffer
+      i_buffer = std::get<0>(m_slice_to_buffer[slice_index]);
+      auto& status = m_buffer_status[i_buffer];
+      m_slice_to_buffer[slice_index] = {-1, 0, 0};
+
+      // If MEPs are not transposed and the respective buffer is no
+      // longer in use, set it to writable
+      if (
+        status.work_counter == 0 &&
+        (std::find_if(m_slice_to_buffer.begin(), m_slice_to_buffer.end(), [i_buffer](const auto& entry) {
+           return std::get<0>(entry) == i_buffer;
+         }) == m_slice_to_buffer.end())) {
+        status.writable = true;
+        set_writable = true;
+      }
+    }
+  }
+  if (freed) {
+    this->debug_output("Freed slice " + std::to_string(slice_index));
+    m_slice_cond.notify_one();
+  }
+  if (set_writable) {
+    this->debug_output("Set buffer " + std::to_string(i_buffer) + " writable");
+    m_mpi_cond.notify_one();
+  }
+}
+
+void MEPProvider::event_sizes(
+  size_t const slice_index,
+  gsl::span<unsigned int const> const selected_events,
+  std::vector<size_t>& sizes) const
+{
+  int i_buffer = 0;
+  size_t interval_start = 0, interval_end = 0;
+  std::tie(i_buffer, interval_start, interval_end) = m_slice_to_buffer[slice_index];
+  auto const& blocks = m_net_slices[i_buffer].blocks;
+  for (unsigned int i = 0; i < selected_events.size(); ++i) {
+    auto event = selected_events[i];
+    sizes[i] +=
+      std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto& entry) {
+        auto const& block_header = std::get<0>(entry);
+        return s + bank_header_size + block_header->bank_sizes()[interval_start + event];
+      });
+  }
+}
+
+void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event, gsl::span<char> buffer) const
+{
+  auto [i_buffer, interval_start, interval_end] = m_slice_to_buffer[slice_index];
+  const auto mep_event = interval_start + event;
+
+  auto const& slice = m_net_slices[i_buffer];
+
+  unsigned char prev_type = 0;
+  auto block_index = 0;
+  size_t offset = 0;
+
+  for (size_t i_block = 0; i_block < slice.blocks.size(); ++i_block) {
+    auto const& [block_header, block_data] = slice.blocks[i_block];
+    auto lhcb_type = block_header->bank_types()[0];
+
+    if (prev_type != lhcb_type) {
+      block_index = 0;
+      prev_type = lhcb_type;
+    }
+
+    // All banks are taken directly from the block data to be able
+    // to treat banks needed by Allen and banks not needed by Allen
+    // in the same way
+    auto const fragment_offset = slice.offsets[i_block][mep_event];
+    auto fragment_size = block_header->bank_sizes()[mep_event];
+
+    assert((offset + fragment_size) < static_cast<size_t>(buffer.size()));
+    offset += add_raw_bank(
+      block_header->bank_types()[mep_event],
+      block_header->block_version,
+      slice.mep_header->src_ids()[i_block],
+      {block_data.data() + fragment_offset, fragment_size},
+      buffer.data() + offset);
+    ++block_index;
+  }
+}
+
+int MEPProvider::start()
+{
+  if (!m_started) {
+    std::unique_lock<std::mutex> lock {m_control_mutex};
+    this->debug_output("Starting", 0);
+    m_started = true;
+    m_stopping = false;
+  }
+  m_control_cond.notify_one();
+  return true;
+};
+
+int MEPProvider::stop()
+{
+  {
+    std::unique_lock<std::mutex> lock {m_control_mutex};
+    m_stopping = true;
+    m_started = false;
+  }
+  // Make sure all threads wait for start in case they were waiting
+  // for a buffer
+  m_mpi_cond.notify_all();
+
+  return true;
+};
+
+void MEPProvider::init_mpi()
+{
+  #ifdef HAVE_MPI
+  auto const& receivers = m_config.receivers;
+  m_domains.reserve(receivers.size());
+
+  // Allocate and initialize topology object.
+  hwloc_topology_init(&m_topology);
+
+  // discover everything, in particular I/O devices like
+  // InfiniBand cards
+  #if HWLOC_API_VERSION >= 0x20000
+  hwloc_topology_set_io_types_filter(m_topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT);
+  #else
+  hwloc_topology_set_flags(m_topology, HWLOC_TOPOLOGY_FLAG_WHOLE_SYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES);
+  #endif
+  // Perform the topology detection.
+  hwloc_topology_load(m_topology);
+
+  hwloc_obj_t osdev = nullptr;
+
+  if (!receivers.empty()) {
+    // Find NUMA domain of receivers
+    while ((osdev = hwloc_get_next_osdev(m_topology, osdev))) {
+      // We're interested in InfiniBand cards
+      if (osdev->attr->osdev.type == HWLOC_OBJ_OSDEV_OPENFABRICS) {
+        auto parent = hwloc_get_non_io_ancestor_obj(m_topology, osdev);
+        auto it = receivers.find(osdev->name);
+        if (it != receivers.end()) {
+          m_domains.emplace_back(it->second, parent->os_index);
+          this->debug_output(
+            "Located receiver device "s + it->first + " in NUMA domain " + std::to_string(parent->os_index));
+        }
+      }
+    }
+    if (m_domains.size() != receivers.size()) {
+      throw StrException {"Failed to locate some receiver devices "};
+    }
+  }
+  else {
+    throw StrException {"MPI requested, but no receivers specified"};
+  }
+
+  // Get last node. There's always at least one.
+  [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
+  assert(static_cast<size_t>(n_numa) == m_domains.size());
+
+  std::vector<hwloc_obj_t> numa_objs(m_config.n_receivers());
+  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
+    int numa_node = std::get<1>(m_domains[receiver]);
+    numa_objs[receiver] = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
+  }
+
+  std::vector<size_t> packing_factors(m_config.n_receivers());
+  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
+    auto const receiver_rank = std::get<0>(m_domains[receiver]);
+    MPI_Recv(
+      &packing_factors[receiver],
+      1,
+      MPI_SIZE_T,
+      receiver_rank,
+      MPI::message::packing_factor,
+      MPI_COMM_WORLD,
+      MPI_STATUS_IGNORE);
+  }
+
+  if (!std::all_of(packing_factors.begin(), packing_factors.end(), [v = packing_factors.back()](auto const p) {
+        return p == v;
+      })) {
+    throw StrException {"All packing factors must be the same"};
+  }
+  else {
+    m_packing_factor = packing_factors.back();
+  }
+
+  // Allocate as many net slices as configured, of expected size
+  // Packing factor can be done dynamically if needed
+  size_t n_bytes = std::lround(m_packing_factor * average_event_size * bank_size_fudge_factor * kB);
+  for (size_t i = 0; i < m_config.n_buffers; ++i) {
+    char* contents = nullptr;
+    MPI_Alloc_mem(n_bytes, MPI_INFO_NULL, &contents);
+
+    // Only bind explicitly if there are multiple receivers,
+    // otherwise assume a memory allocation policy is in effect
+    if (m_domains.size() > 1) {
+      auto const& numa_obj = numa_objs[numa_node];
+      auto s = hwloc_set_area_membind(
+        m_topology, contents, n_bytes, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
+      if (s != 0) {
+        throw StrException {"Failed to bind memory to node "s + std::to_string(numa_obj->os_index) + " " +
+                            strerror(errno)};
+      }
+    }
+
+    cudaCheck(cudaHostRegister(contents, n_bytes, cudaHostRegisterDefault));
+    m_net_slices.emplace_back(
+      nullptr,
+      0u,
+      gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
+      MEP::Blocks {},
+      MEP::SourceOffsets {},
+      n_bytes);
+    m_mpi_buffers.emplace_back(contents);
+  }
+  #else
+  error_cout << "MPI requested, but Allen was not built with MPI support.\n";
+  throw std::runtime_error {"No MPI supoprt"};
+  #endif
+}
+
+size_t MEPProvider::count_writable() const
+{
+  return std::accumulate(m_buffer_status.begin(), m_buffer_status.end(), 0ul, [](size_t s, BufferStatus const& stat) {
+    return s + stat.writable;
+  });
+}
+
+void MEPProvider::allocate_storage(size_t i_read)
+{
+  if (m_sizes_known) return;
+
+  // Count number of banks per flavour
+  bool count_success = false;
+
+  // Offsets are to the start of the event, which includes the header
+  auto& slice = m_net_slices[i_read];
+  auto const* mep_header = slice.mep_header;
+  size_t n_blocks = mep_header->n_MFPs;
+
+  if (m_packing_factor == 0) {
+    m_packing_factor = slice.packing_factor;
+  }
+  else {
+    assert(slice.packing_factor == m_packing_factor);
+  }
+
+  auto eps = this->events_per_slice();
+  auto n_interval = m_packing_factor / eps;
+  auto rest = m_packing_factor % eps;
+  for (auto& s : m_buffer_status) {
+    s.intervals.reserve(2 * (n_interval + rest));
+  }
+
+  for (auto& slice : m_net_slices) {
+    // The number of blocks in a MEP is known, use it to allocate
+    // temporary storage used during transposition
+    slice.blocks.resize(n_blocks);
+    slice.offsets.resize(n_blocks);
+    for (auto& offsets : slice.offsets) {
+      // info_cout << "Packing factor: " << mep_header->packing_factor << "\n";
+      offsets.resize(m_packing_factor + 1);
+    }
+  }
+
+  std::tie(count_success, m_banks_count) = MEP::fill_counts(mep_header, slice.mep_data);
+
+  // Allocate slice memory that will contain transposed banks ready
+  // for processing by the Allen kernels
+  auto size_fun = [this, eps](BankTypes bank_type) -> std::tuple<size_t, size_t> {
+    auto it = BankSizes.find(bank_type);
+    auto ib = to_integral(bank_type);
+    if (it == end(BankSizes)) {
+      throw std::out_of_range {std::string {"Bank type "} + std::to_string(ib) + " has no known size"};
+    }
+    // In case of direct MEP output, no memory should be allocated.
+    if (!m_config.transpose_mep) {
+      auto it = std::find(m_bank_ids.begin(), m_bank_ids.end(), to_integral(bank_type));
+      auto lhcb_type = std::distance(m_bank_ids.begin(), it);
+      auto n_blocks = m_banks_count[lhcb_type];
+      // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
+      return {0, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
+    }
+    else {
+      auto aps = eps < 100 ? 100 : eps;
+      return {std::lround(it->second * aps * bank_size_fudge_factor * kB), eps};
+    }
+  };
+  m_slices = allocate_slices(this->n_slices(), types(), size_fun);
+
+  m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(this->n_slices(), std::make_tuple(-1, 0ul, 0ul));
+
+  if (!count_success) {
+    error_cout << "Failed to determine bank counts\n";
+    m_read_error = true;
+  }
+  else {
+    m_sizes_known = true;
+  }
+}
+
+bool MEPProvider::open_file() const
+{
+  bool good = false;
+
+  // Check if there are still files available
+  while (!good) {
+    // If looping on input is configured, do it
+    if (m_current == m_connections.end()) {
+      if (m_config.non_stop) {
+        m_current = m_connections.begin();
+      }
+      else {
+        break;
+      }
+    }
+
+    if (m_input) m_input->close();
+
+    m_input = LHCb::StreamDescriptor::connect(*m_current);
+    if (m_input->ioDesc != 0) {
+      info_cout << "Opened " << *m_current << "\n";
+      good = true;
+    }
+    else {
+      error_cout << "Failed to open " << *m_current << " " << strerror(errno) << "\n";
+      m_read_error = true;
+      return false;
+    }
+    ++m_current;
+  }
+  return good;
+}
+
+std::tuple<std::vector<BufferStatus>::iterator, size_t> MEPProvider::get_mep_buffer(
+  std::function<bool(BufferStatus const&)> pred,
+  std::vector<BufferStatus>::iterator start,
+  std::unique_lock<std::mutex>& lock)
+{
+  // Obtain a prefetch buffer to read into, if none is available,
+  // wait until one of the transpose threads is done with its
+  // prefetch buffer
+  auto find_buffer = [this, start, &pred] {
+    auto it = std::find_if(start, m_buffer_status.end(), pred);
+    if (it == m_buffer_status.end()) {
+      it = std::find_if(m_buffer_status.begin(), start, pred);
+      if (it == start) it = m_buffer_status.end();
+    }
+    return it;
+  };
+
+  auto it = find_buffer();
+  if (it == m_buffer_status.end() && !m_transpose_done) {
+    m_mpi_cond.wait(lock, [this, &it, &find_buffer] {
+      it = find_buffer();
+      return it != m_buffer_status.end() || m_transpose_done || m_stopping;
+    });
+  }
+  return {it, distance(m_buffer_status.begin(), it)};
+}
+
+void MEPProvider::set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events)
+{
+  if (n_events == 0) return;
+  const auto eps = this->events_per_slice();
+  auto n_interval = n_events / eps;
+  auto rest = n_events % eps;
+  if (rest) {
+    debug_cout << "Set interval (rest): " << n_interval * eps << "," << n_interval * eps + rest << "\n";
+    intervals.emplace_back(n_interval * eps, n_interval * eps + rest);
+  }
+  for (size_t i = n_interval; i != 0; --i) {
+    debug_cout << "Set interval: " << (i - 1) * eps << "," << i * eps << "\n";
+    intervals.emplace_back((i - 1) * eps, i * eps);
+  }
+}
+
+// mep reader thread
+void MEPProvider::mep_read()
+{
+  bool receive_done = false;
+
+  auto to_read = this->n_events();
+  if (to_read) debug_cout << "Reading " << *to_read << " events\n";
+  auto to_publish = 0;
+
+  while (!receive_done) {
+    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
+
+    // If we've been stopped, wait for start or exit
+    if (!m_started || m_stopping) {
+      std::unique_lock<std::mutex> lock {m_control_mutex};
+      this->debug_output("Waiting for start", 0);
+      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+    }
+
+    if (m_done) break;
+
+    // open the first file
+    if (!m_input && !open_file()) {
+      m_read_error = true;
+      m_mpi_cond.notify_one();
+      return;
+    }
+    size_t i_buffer;
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::tie(m_buffer_reading, i_buffer) =
+        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+      if (m_buffer_reading != m_buffer_status.end()) {
+        m_buffer_reading->writable = false;
+        assert(m_buffer_reading->work_counter == 0);
+      }
+      else {
+        continue;
+      }
+    }
+    if (m_done) {
+      receive_done = true;
+      break;
+    }
+
+    this->debug_output("Writing to MEP slice index " + std::to_string(i_buffer));
+
+    auto& read_buffer = m_read_buffers[i_buffer];
+    auto& slice = m_net_slices[i_buffer];
+
+    bool success = false, eof = false;
+
+    while (!success || eof) {
+      std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
+
+      if (!eof) {
+        debug_cout << "Read mep with packing factor " << slice.packing_factor << "\n";
+        if (to_read && success) {
+          to_publish = std::min(*to_read, size_t {slice.packing_factor});
+          *to_read -= to_publish;
+        }
+        else {
+          to_publish = slice.packing_factor;
+        }
+      }
+
+      if (!success) {
+        // Error encountered
+        m_read_error = true;
+        break;
+      }
+      else if ((to_read && *to_read == 0) || (eof && !open_file())) {
+        // Try to open the next file, if there is none, prefetching
+        // is done.
+        if (!m_read_error) {
+          this->debug_output("Prefetch done: eof and no more files");
+        }
+        receive_done = true;
+        break;
+      }
+    }
+
+    if (!m_sizes_known) {
+      allocate_storage(i_buffer);
+    }
+
+    assert(slice.packing_factor = m_packing_factor);
+
+    // Notify a transpose thread that a new buffer of events is
+    // ready. If prefetching is done, wake up all threads
+    if (success) {
+      {
+        std::unique_lock<std::mutex> lock {m_mpi_mutex};
+
+        auto& status = m_buffer_status[i_buffer];
+        assert(status.work_counter == 0);
+
+        if (!eof && to_publish != 0) {
+          set_intervals(status.intervals, to_read ? to_publish : size_t {slice.packing_factor});
+        }
+        else {
+          // We didn't read anything, so free the buffer we got again
+          status.writable = true;
+        }
+      }
+      if (receive_done) {
+        m_done = receive_done;
+        this->debug_output("Prefetch notifying all");
+        m_mpi_cond.notify_all();
+      }
+      else if (!eof) {
+        this->debug_output("Prefetch notifying one");
+        m_mpi_cond.notify_one();
+      }
+    }
+    m_mpi_cond.notify_one();
+  }
+}
+
+// MPI reader thread
+void MEPProvider::mpi_read()
+{
+  #ifdef HAVE_MPI
+  int window_size = m_config.window_size;
+  std::vector<MPI_Request> requests(window_size);
+
+  // Iterate over the slices
+  size_t reporting_period = 5;
+  std::vector<std::tuple<size_t, size_t>> data_received(m_config.n_receivers());
+  std::vector<size_t> n_meps(m_config.n_receivers());
+  Timer t;
+  Timer t_origin;
+  bool error = false;
+
+  for (size_t i = 0; i < m_config.n_receivers(); ++i) {
+    auto [mpi_rank, numa_domain] = m_domains[i];
+    MPI_Recv(&n_meps[i], 1, MPI_SIZE_T, mpi_rank, MPI::message::number_of_meps, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
+  }
+  size_t number_of_meps = std::accumulate(n_meps.begin(), n_meps.end(), 0u);
+
+  size_t current_mep = 0;
+  while (!m_done && (m_config.non_stop || current_mep < number_of_meps)) {
+    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
+
+    // If we've been stopped, wait for start or exit
+    if (!m_started || m_stopping) {
+      std::unique_lock<std::mutex> lock {m_control_mutex};
+      this->debug_output("Waiting for start", 0);
+      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+    }
+
+    if (m_done) break;
+
+    // Obtain a prefetch buffer to read into, if none is available,
+    // wait until one of the transpose threads is done with its
+    // prefetch buffer
+    size_t i_buffer;
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::tie(m_buffer_reading, i_buffer) =
+        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+      if (m_buffer_reading != m_buffer_status.end()) {
+        m_buffer_reading->writable = false;
+        assert(m_buffer_reading->work_counter == 0);
+      }
+      else {
+        continue;
+      }
+    }
+
+    auto receiver = i_buffer % m_config.n_receivers();
+    auto [sender_rank, numa_node] = m_domains[receiver];
+
+    this->debug_output(
+      "Receiving from rank " + std::to_string(sender_rank) + " into buffer " + std::to_string(i_buffer) +
+      "  NUMA domain " + std::to_string(numa_node));
+
+    auto& slice = m_net_slices[i_buffer];
+    char*& contents = m_mpi_buffers[i_buffer];
+
+    size_t mep_size = 0;
+    MPI_Recv(&mep_size, 1, MPI_SIZE_T, sender_rank, MPI::message::event_size, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
+
+    // Reallocate if needed
+    if (mep_size > buffer_size) {
+      buffer_size = mep_size * bank_size_fudge_factor;
+      // Unregister memory
+      cudaCheck(cudaHostUnregister(contents));
+
+      // Free memory
+      MPI_Free_mem(contents);
+
+      // Allocate new memory
+      MPI_Alloc_mem(buffer_size, MPI_INFO_NULL, &contents);
+
+      // Only bind explicitly if there are multiple receivers,
+      // otherwise assume a memory allocation policy is in effect
+      if (m_domains.size() > 1) {
+        // Bind memory to numa domain of receiving card
+        auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
+        auto s = hwloc_set_area_membind(
+          m_topology, contents, buffer_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
+        if (s != 0) {
+          m_read_error = true;
+          error_cout << "Failed to bind memory to node " << std::to_string(numa_node) << " " << strerror(errno) << "\n";
+          break;
+        }
+      }
+
+      // Register memory with CUDA
+      try {
+        cudaCheck(cudaHostRegister(contents, buffer_size, cudaHostRegisterDefault));
+      } catch (std::invalid_argument const&) {
+        m_read_error = true;
+        break;
+      }
+
+      buffer_span = gsl::span {contents, static_cast<events_size>(buffer_size)};
+    }
+
+    // Number of full-size (MPI::mdf_chunk_size) messages
+    int n_messages = mep_size / MPI::mdf_chunk_size;
+    // Size of the last message (if the MFP size is not a multiple of MPI::mdf_chunk_size)
+    int rest = mep_size - n_messages * MPI::mdf_chunk_size;
+    // Number of parallel sends
+    int n_sends = n_messages > window_size ? window_size : n_messages;
+
+    // info_cout << MPI::rank_str() << "n_messages " << n_messages << ", rest " << rest << ", n_sends " << n_sends <<
+    // "\n";
+
+    // Initial parallel sends
+    for (int k = 0; k < n_sends; k++) {
+      char* message = contents + k * MPI::mdf_chunk_size;
+      MPI_Irecv(
+        message,
+        MPI::mdf_chunk_size,
+        MPI_BYTE,
+        sender_rank,
+        MPI::message::event_send_tag_start + k,
+        MPI_COMM_WORLD,
+        &requests[k]);
+    }
+    // Sliding window sends
+    for (int k = n_sends; k < n_messages; k++) {
+      int r;
+      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      char* message = contents + k * MPI::mdf_chunk_size;
+      MPI_Irecv(
+        message,
+        MPI::mdf_chunk_size,
+        MPI_BYTE,
+        sender_rank,
+        MPI::message::event_send_tag_start + k,
+        MPI_COMM_WORLD,
+        &requests[r]);
+    }
+    // Last send (if necessary)
+    if (rest) {
+      int r;
+      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      char* message = contents + n_messages * MPI::mdf_chunk_size;
+      MPI_Irecv(
+        message,
+        rest,
+        MPI_BYTE,
+        sender_rank,
+        MPI::message::event_send_tag_start + n_messages,
+        MPI_COMM_WORLD,
+        &requests[r]);
+    }
+    // Wait until all chunks have been sent
+    MPI_Waitall(n_sends, requests.data(), MPI_STATUSES_IGNORE);
+
+    slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(contents);
+    slice.mep_data = gsl::span {contents, static_cast<events_size>(mep_size)};
+
+    size_t n_blocks = mep_header->n_MFPs;
+    auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0]);
+    slice.packing_factor = block_header->n_banks;
+
+    if (!m_sizes_known) {
+      allocate_storage(i_buffer);
+    }
+
+    auto& [meps_received, bytes_received] = data_received[receiver];
+    bytes_received += mep_size;
+    meps_received += 1;
+    if (t.get_elapsed_time() >= reporting_period) {
+      const auto seconds = t.get_elapsed_time();
+      auto total_rate = 0.;
+      auto total_bandwidth = 0.;
+      for (size_t i_rec = 0; i_rec < m_config.n_receivers(); ++i_rec) {
+        auto& [mr, br] = data_received[i_rec];
+        auto [rec_rank, rec_node] = m_domains[i_rec];
+
+        const double rate = (double) mr / seconds;
+        const double bandwidth = ((double) (br * 8)) / (1024 * 1024 * 1024 * seconds);
+        total_rate += rate;
+        total_bandwidth += bandwidth;
+        printf(
+          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s; Domain %2i; Rank %2i\n",
+          t_origin.get_elapsed_time(),
+          seconds,
+          rate,
+          bandwidth,
+          rec_node,
+          rec_rank);
+
+        br = 0;
+        mr = 0;
+      }
+      if (m_config.n_receivers() > 1) {
+        printf(
+          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s\n",
+          t_origin.get_elapsed_time(),
+          seconds,
+          total_rate,
+          total_bandwidth);
+      }
+      t.restart();
+    }
+
+    // Notify a transpose thread that a new buffer of events is
+    // ready. If prefetching is done, wake up all threads
+    if (!error) {
+      {
+        std::unique_lock<std::mutex> lock {m_mpi_mutex};
+        set_intervals(m_buffer_status[i_buffer].intervals, size_t {slice.packing_factor});
+        assert(m_buffer_status[i_buffer].work_counter == 0);
+      }
+      this->debug_output("Prefetch notifying one");
+      m_mpi_cond.notify_one();
+    }
+    m_mpi_cond.notify_one();
+
+    current_mep++;
+  }
+
+  if (!m_done) {
+    m_done = true;
+    this->debug_output("Prefetch notifying all");
+    m_mpi_cond.notify_all();
+  }
+  #endif
+}
+
+
+/**
+ * @brief      Function to run in each thread transposing events
+ *
+ * @param      thread ID
+ *
+ * @return     void
+ */
+void MEPProvider::transpose(int thread_id)
+{
+
+  size_t i_buffer = 0;
+  std::tuple<size_t, size_t> interval;
+  std::optional<size_t> slice_index;
+
+  bool good = false, transpose_full = false;
+  size_t n_transposed = 0;
+
+  auto has_intervals = [](BufferStatus const& s) { return !s.intervals.empty(); };
+
+  while (!m_read_error && !m_transpose_done) {
+    // Get a buffer to read from
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::tie(m_buffer_transpose, i_buffer) = get_mep_buffer(has_intervals, m_buffer_transpose, lock);
+      if (m_transpose_done) {
+        break;
+      }
+      else if (m_buffer_transpose == m_buffer_status.end()) {
+        continue;
+      }
+      auto& status = *m_buffer_transpose;
+      assert(!status.intervals.empty());
+
+      interval = status.intervals.back();
+      status.intervals.pop_back();
+
+      ++(status.work_counter);
+      status.writable = false;
+
+      this->debug_output(
+        "Got MEP slice index " + std::to_string(i_buffer) + " interval [" + std::to_string(std::get<0>(interval)) +
+          "," + std::to_string(std::get<1>(interval)) + ")",
+        thread_id);
+    }
+
+    // Get a slice to write to
+    if (!slice_index) {
+      this->debug_output("Getting slice index", thread_id);
+      auto it = m_slice_free.end();
+      {
+        std::unique_lock<std::mutex> lock {m_slice_mut};
+        it = find(m_slice_free.begin(), m_slice_free.end(), true);
+        if (it == m_slice_free.end()) {
+          this->debug_output("Waiting for free slice", thread_id);
+          m_slice_cond.wait(lock, [this, &it] {
+            it = std::find(m_slice_free.begin(), m_slice_free.end(), true);
+            return it != m_slice_free.end() || m_transpose_done;
+          });
+          // If transpose is done and there is no slice, we were
+          // woken up be the desctructor before a slice was declared
+          // free. In that case, exit without transposing
+          if (m_transpose_done && it == m_slice_free.end()) {
+            break;
+          }
+        }
+        *it = false;
+        slice_index = distance(m_slice_free.begin(), it);
+        this->debug_output("Got slice index " + std::to_string(*slice_index), thread_id);
+
+        // Keep track of what buffer this slice belonged to
+        m_slice_to_buffer[*slice_index] = {i_buffer, std::get<0>(interval), std::get<1>(interval)};
+      }
+    }
+
+    // Reset the slice
+    auto& event_ids = m_event_ids[*slice_index];
+    //    reset_slice(m_slices, *slice_index, bank_types, event_ids, !m_config.transpose_mep);
+    reset_slice(m_slices, *slice_index, types(), event_ids, !m_config.transpose_mep);
+
+
+
+    // MEP data
+    auto& slice = m_net_slices[i_buffer];
+
+    // Fill blocks
+    MEP::find_blocks(slice.mep_header, slice.packing_factor, slice.mep_data, slice.blocks);
+
+    // Fill fragment offsets
+    MEP::fragment_offsets(slice.blocks, slice.offsets);
+
+    // Transpose or calculate offsets
+    if (m_config.transpose_mep) {
+      // Transpose the events into the slice
+      std::tie(good, transpose_full, n_transposed) = MEP::transpose_events(
+        m_slices,
+        *slice_index,
+        m_bank_ids,
+        this->types(),
+        m_banks_count,
+        event_ids,
+        slice.mep_header,
+        slice.blocks,
+        slice.offsets,
+        interval);
+      this->debug_output(
+        "Transposed slice " + std::to_string(*slice_index) + "; good: " + std::to_string(good) +
+          "; full: " + std::to_string(transpose_full) + "; n_transposed:  " + std::to_string(n_transposed),
+        thread_id);
+    }
+    else {
+      // Calculate fragment offsets in MEP per sub-detector
+      std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
+        m_slices, *slice_index, m_bank_ids, this->types(), m_banks_count, event_ids, slice.mep_header, slice.blocks, interval);
+      this->debug_output("Calculated MEP offsets for slice " + std::to_string(*slice_index), thread_id);
+    }
+
+    if (m_read_error || !good) {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      auto& status = m_buffer_status[i_buffer];
+      --status.work_counter;
+      m_read_error = true;
+      m_transpose_cond.notify_one();
+      break;
+    }
+
+    // Notify any threads waiting in get_slice that a slice is available
+    {
+      std::unique_lock<std::mutex> lock {m_transpose_mut};
+      m_transposed.emplace_back(*slice_index, n_transposed);
+    }
+    m_transpose_cond.notify_one();
+    slice_index.reset();
+
+    // Check if the read buffer is now empty. If it is, it can be
+    // reused, otherwise give it to another transpose thread once a
+    // new target slice is available
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+
+      auto& status = m_buffer_status[i_buffer];
+      --status.work_counter;
+
+      if (n_transposed != std::get<1>(interval) - std::get<0>(interval)) {
+        status.intervals.emplace_back(std::get<0>(interval) + n_transposed, std::get<1>(interval));
+      }
+      else if (status.work_counter == 0) {
+        m_transpose_done =
+          m_done && std::all_of(m_buffer_status.begin(), m_buffer_status.end(), [](BufferStatus const& stat) {
+            return stat.intervals.empty() && stat.work_counter == 0;
+          });
+      }
+    }
+  }
+}
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
new file mode 100644
index 000000000..ce804c6ec
--- /dev/null
+++ b/AllenOnline/src/MEPProvider.h
@@ -0,0 +1,263 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#pragma once
+
+#include <thread>
+#include <vector>
+#include <array>
+#include <deque>
+#include <mutex>
+#include <atomic>
+#include <chrono>
+#include <algorithm>
+#include <numeric>
+#include <condition_variable>
+#include <cassert>
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+
+#include <Event/RawBank.h>
+#include <MDF/MDFHeader.h>
+#include <MDF/StreamDescriptor.h>
+
+#include "TransposeMEP.h"
+
+namespace {
+  using namespace Allen::Units;
+  using namespace std::string_literals;
+
+  constexpr auto bank_header_size = sizeof(LHCb::RawBank) - sizeof(unsigned int);
+} // namespace
+
+/**
+ * @brief      Configuration parameters for the MEPProvider
+ */
+struct MEPProviderConfig {
+  // check the MDF checksum if it is available
+  bool check_checksum = false;
+
+  // number of prefetch buffers
+  size_t n_buffers = 8;
+
+  // number of transpose threads
+  size_t n_transpose_threads = 5;
+
+  int window_size = 4;
+
+  // Use MPI and number of receivers
+  bool mpi = false;
+
+  bool non_stop = true;
+
+  bool transpose_mep = false;
+
+  bool split_by_run = false;
+
+  size_t n_receivers() const { return receivers.size(); }
+
+  // Mapping of receiver card to MPI rank to receive from
+  std::map<std::string, int> receivers;
+};
+
+/**
+ * @brief      Provide transposed events from MDF files
+ *
+ * @details    The provider has three main components
+ *             - a prefetch thread to read from the current input
+ *               file into prefetch buffers
+ *             - N transpose threads that read from prefetch buffers
+ *               and fill the per-bank-type slices with transposed sets
+ *               of banks and the offsets to individual bank inside a
+ *               given set
+ *             - functions to obtain a transposed slice and declare it
+ *               for refilling
+ *
+ *             Access to prefetch buffers and slices is synchronised
+ *             using mutexes and condition variables.
+ *
+ * @param      Number of slices to fill
+ * @param      Number of events per slice
+ * @param      MDF filenames
+ * @param      Configuration struct
+ *
+ */
+class MEPProvider final : public InputProvider {
+public:
+  MEPProvider(
+    size_t n_slices,
+    size_t events_per_slice,
+    std::optional<size_t> n_events,
+    std::vector<std::string> connections,
+    std::unordered_set<BankTypes> const& bank_types,
+    MEPProviderConfig config = MEPProviderConfig {}) noexcept(false);
+
+  /**
+   * @brief      Obtain event IDs of events stored in a given slice
+   *
+   * @param      slice index
+   *
+   * @return     EventIDs of events in given slice
+   */
+  EventIDs event_ids(size_t slice_index, std::optional<size_t> first = {}, std::optional<size_t> last = {})
+    const override;
+
+  /**
+   * @brief      Obtain banks from a slice
+   *
+   * @param      BankType
+   * @param      slice index
+   *
+   * @return     Banks and their offsets
+   */
+  BanksAndOffsets banks(BankTypes bank_type, size_t slice_index) const override;
+
+  /**
+   * @brief      Get a slice that is ready for processing; thread-safe
+   *
+   * @param      optional timeout
+   *
+   * @return     (good slice, timed out, slice index, number of events in slice)
+   */
+  std::tuple<bool, bool, bool, size_t, size_t, uint> get_slice(
+    std::optional<unsigned int> timeout = {}) override;
+
+  /**
+   * @brief      Declare a slice free for reuse; thread-safe
+   *
+   * @param      slice index
+   *
+   * @return     void
+   */
+  void slice_free(size_t slice_index) override;
+
+  void event_sizes(
+    size_t const slice_index,
+    gsl::span<unsigned int const> const selected_events,
+    std::vector<size_t>& sizes) const override;
+
+  void copy_banks(size_t const slice_index, unsigned int const event, gsl::span<char> buffer) const override;
+
+  int start() override;
+
+  int stop() override;
+
+private:
+  void init_mpi();
+
+  size_t count_writable() const;
+
+  void allocate_storage(size_t i_read);
+
+  /**
+   * @brief      Open an input file; called from the prefetch thread
+   *
+   * @return     success
+   */
+  bool open_file() const;
+
+  std::tuple<std::vector<BufferStatus>::iterator, size_t> get_mep_buffer(
+    std::function<bool(BufferStatus const&)> pred,
+    std::vector<BufferStatus>::iterator start,
+    std::unique_lock<std::mutex>& lock);
+
+  void set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events);
+
+  // mep reader thread
+  void mep_read();
+
+  // MPI reader thread
+  void mpi_read();
+
+  /**
+   * @brief      Function to run in each thread transposing events
+   *
+   * @param      thread ID
+   *
+   * @return     void
+   */
+  void transpose(int thread_id);
+
+  // Slices
+  size_t m_packing_factor = 0;
+  std::vector<std::vector<char>> m_read_buffers;
+  std::vector<char*> m_mpi_buffers;
+  MEP::Slices m_net_slices;
+
+  // data members for mpi thread
+  bool m_started = false;
+  bool m_stopping = false;
+  std::mutex m_control_mutex;
+  std::condition_variable m_control_cond;
+
+  // data members for mpi thread
+  std::mutex m_mpi_mutex;
+  std::condition_variable m_mpi_cond;
+
+  #ifdef HAVE_MPI
+  std::vector<std::tuple<int, int>> m_domains;
+  #endif
+
+  #ifdef HAVE_HWLOC
+  hwloc_topology_t m_topology;
+  #endif
+
+  std::vector<BufferStatus> m_buffer_status;
+  std::vector<BufferStatus>::iterator m_buffer_transpose;
+  std::vector<BufferStatus>::iterator m_buffer_reading;
+  std::thread m_input_thread;
+
+  // Atomics to flag errors and completion
+  std::atomic<bool> m_done = false;
+  mutable std::atomic<bool> m_read_error = false;
+  std::atomic<bool> m_transpose_done = false;
+
+  // Buffer to store data read from file if banks are compressed. The
+  // decompressed data will be written to the buffers
+  mutable std::vector<char> m_compress_buffer;
+
+  // Storage to read the header into for each event
+  mutable LHCb::MDFHeader m_header;
+
+  // Allen IDs of LHCb raw banks
+  std::vector<int> m_bank_ids;
+
+  // Memory slices, N for each raw bank type
+  Slices m_slices;
+  std::vector<std::tuple<int, size_t, size_t>> m_slice_to_buffer;
+
+  // Mutex, condition varaible and queue for parallel transposition of slices
+  std::mutex m_transpose_mut;
+  std::condition_variable m_transpose_cond;
+  std::deque<std::tuple<size_t, size_t>> m_transposed;
+
+  // Keep track of what slices are free
+  std::mutex m_slice_mut;
+  std::condition_variable m_slice_cond;
+  std::vector<bool> m_slice_free;
+
+  // Threads transposing data
+  std::vector<std::thread> m_transpose_threads;
+
+  // Array to store the number of banks per bank type
+  mutable std::array<unsigned int, LHCb::NBankTypes> m_banks_count;
+  mutable bool m_sizes_known = false;
+
+  // Run and event numbers present in each slice
+  std::vector<EventIDs> m_event_ids;
+
+  // File names to read
+  std::vector<std::string> m_connections;
+
+  // Storage for the currently open input file
+  mutable std::optional<LHCb::StreamDescriptor::Access> m_input;
+
+  // Iterator that points to the filename of the currently open file
+  mutable std::vector<std::string>::const_iterator m_current;
+
+  // Configuration struct
+  MEPProviderConfig m_config;
+};
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
new file mode 100644
index 000000000..acbaecbf7
--- /dev/null
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -0,0 +1,97 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+
+#include <iostream>
+#include <vector>
+#include <cstring>
+#include <cassert>
+
+#include <gsl/gsl>
+
+#include <EventBuilding/MEP_tools.hpp>
+#include <EventBuilding/MFP_tools.hpp>
+#include <MDF/MDFHeader.h>
+#include <MDF/PosixIO.h>
+#include <Allen/BankTypes.h>
+
+#include "ReadMEP.h"
+
+namespace {
+  using std::cerr;
+  using std::cout;
+} // namespace
+
+/**
+ * @brief      Read a mep from a file
+ *
+ * @param      file descriptor to read from
+ * @param      buffer to store data in
+ *
+ * @return     (eof, success, mep_header, span of mep data)
+ */
+std::tuple<bool, bool, MEP::MEP_header const*, unsigned, gsl::span<char const>>
+MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
+{
+
+  buffer.resize(sizeof(LHCb::MDFHeader));
+  LHCb::MDFHeader* mdf_header = reinterpret_cast<LHCb::MDFHeader*>(buffer.data());
+
+  ssize_t n_bytes = input.ioFuncs->read(input.ioDesc, &buffer[0], sizeof(LHCb::MDFHeader));
+  if (n_bytes == 0) {
+    cout << "Cannot read more data (Header). End-of-File reached.\n";
+    return {true, true, nullptr, 0u, {}};
+  }
+  else if (n_bytes < 0) {
+    cerr << "Failed to read header " << strerror(errno) << "\n";
+    return {false, false, nullptr, 0u, {}};
+  }
+  unsigned header_version = mdf_header->headerVersion();
+  auto hdr_size = LHCb::MDFHeader::sizeOf(header_version);
+  assert((hdr_size - sizeof(LHCb::MDFHeader)) == mdf_header->subheaderLength());
+  // read subheader
+  buffer.resize(hdr_size + sizeof(MEP::MEP_header));
+  mdf_header = reinterpret_cast<LHCb::MDFHeader*>(&buffer[0]);
+  n_bytes = input.read(&buffer[0] + sizeof(LHCb::MDFHeader), mdf_header->subheaderLength());
+  if (n_bytes <= 0) {
+    cerr << "Failed to read subheader " << strerror(errno) << "\n";
+    return {false, false, nullptr, 0u, {}};
+  }
+
+  // read MEP_header
+  char* mep_buffer = &buffer[0] + hdr_size;
+  MEP::MEP_header* mep_header = reinterpret_cast<MEP::MEP_header*>(mep_buffer);
+  n_bytes = input.read(mep_buffer, sizeof(MEP::MEP_header));
+  if (n_bytes <= 0) {
+    cerr << "Failed to MEP_header base " << strerror(errno) << "\n";
+    return {false, false, nullptr, 0u, {}};
+  }
+
+  buffer.resize(hdr_size + mep_header->header_size());
+  mep_buffer = &buffer[0] + hdr_size;
+  mep_header = reinterpret_cast<MEP::MEP_header*>(mep_buffer);
+  auto data_size = static_cast<size_t>(mep_header->bytes());
+
+  buffer.resize(hdr_size + mep_header->header_size() + data_size);
+  mdf_header = reinterpret_cast<LHCb::MDFHeader*>(&buffer[0]);
+  mep_buffer = &buffer[0] + hdr_size;
+  mep_header = reinterpret_cast<MEP::MEP_header*>(mep_buffer);
+
+  n_bytes = input.ioFuncs->read(input.ioDesc,
+    mep_buffer + mep_header->header_size(),
+    static_cast<int>(mep_header->header_size() - sizeof(MEP::MEP_header) + data_size));
+  if (n_bytes <= 0) {
+    cerr << "Failed to read MEP" << strerror(errno) << "\n";
+    return {false, false, nullptr, 0u, {}};
+  }
+
+  auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0]);
+
+  auto total_size = mep_header->header_size() + data_size;
+  return {false, true, mep_header, block_header->n_banks,
+          {buffer.data() + hdr_size, total_size}};
+}
diff --git a/AllenOnline/src/ReadMEP.h b/AllenOnline/src/ReadMEP.h
new file mode 100644
index 000000000..47a1f25cf
--- /dev/null
+++ b/AllenOnline/src/ReadMEP.h
@@ -0,0 +1,16 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#pragma once
+
+#include <vector>
+
+#include <gsl/gsl>
+
+#include <EventBuilding/MEP_tools.hpp>
+#include <MDF/StreamDescriptor.h>
+
+namespace MEP {
+  std::tuple<bool, bool, MEP::MEP_header const*, unsigned, gsl::span<char const>>
+  read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer);
+}
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
new file mode 100644
index 000000000..9503682fb
--- /dev/null
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -0,0 +1,388 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <cassert>
+#include <cstring>
+#include "TransposeMEP.h"
+
+#include <Event/ODIN.h>
+
+namespace {
+  std::tuple<unsigned, unsigned long> decode_odin(unsigned const* odin_data) {
+    unsigned const run_number = odin_data[LHCb::ODIN::Data::RunNumber];
+    unsigned long evt_hi = odin_data[LHCb::ODIN::Data::L0EventIDHi];
+    unsigned long evt_lo = odin_data[LHCb::ODIN::Data::L0EventIDLo];
+    unsigned long event_number = (evt_hi << 32) | evt_lo;
+    return {run_number, event_number};
+  }
+}
+
+std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>> MEP::fill_counts(
+  MEP::MEP_header const* header,
+  gsl::span<char const> const& mep_span)
+{
+  // info_cout << "EB header: "
+  //   << header.n_blocks << ", "
+  //   << header.packing_factor << ", "
+  //   << header.reserved << ", "
+  //   << header.mep_size << "\n";
+
+  auto header_size = header->header_size();
+  gsl::span<char const> block_span {mep_span.data() + header_size, mep_span.size() - header_size};
+  std::array<unsigned int, LHCb::NBankTypes> count {0};
+  for (size_t i = 0; i < header->n_MFPs; ++i) {
+    auto offset = header->offsets()[i];
+    auto const* bh = reinterpret_cast<MFP::MFP_header const*>(block_span.data() + offset);
+
+    // info_cout << "EB BlockHeader: "
+    //   << bh.event_id << ", " << bh.n_frag << ", " << bh.reserved << ", " << bh.block_size << "\n";
+
+    assert(bh->n_frag != 0);
+    auto type = bh->bank_types()[0];
+    if (type < LHCb::RawBank::LastType) {
+      ++count[type];
+    }
+  }
+
+  return {true, count};
+}
+
+void MEP::find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor, gsl::span<char const> const& buffer_span, Blocks& blocks)
+{
+
+  // Fill blocks
+  auto hdr_size = mep_header->header_size();
+  auto block_hdr_size = MFP::MFP_header_size(packing_factor, MFP::MFP_aligment);
+  gsl::span<char const> const mep_data {buffer_span.data() + hdr_size, buffer_span.size() - hdr_size};
+
+  for (size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
+    auto block_offset = mep_header->offsets()[i_block];
+    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>(mep_data.data() + block_offset);
+    gsl::span<char const> block_data {mep_data.data() + block_offset + block_hdr_size, block_header->bytes()};
+    blocks[i_block] = std::tuple {block_header, std::move(block_data)};
+  }
+}
+
+void MEP::fragment_offsets(MEP::Blocks const& blocks, MEP::SourceOffsets& offsets)
+{
+
+  // Reset input offsets
+  for (auto& o : offsets) {
+    std::fill(o.begin(), o.end(), 0);
+  }
+
+  // Loop over all bank sizes in all blocks
+  for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
+    auto const& [block_header, block_data] = blocks[i_block];
+    auto& o = offsets[i_block];
+    uint32_t fragment_offset = 0;
+
+    for (size_t i = 0; i < block_header->n_banks; ++i) {
+      o[i] = fragment_offset;
+      fragment_offset += block_header->bank_sizes()[i];
+    }
+  }
+}
+
+size_t MEP::allen_offsets(
+  ::Slices& slices,
+  int const slice_index,
+  std::vector<int> const& bank_ids,
+  std::unordered_set<BankTypes> const& bank_types,
+  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+  MEP::Blocks const& blocks,
+  MEP::SourceOffsets const& input_offsets,
+  std::tuple<size_t, size_t> const& interval,
+  bool split_by_run)
+{
+
+  auto [event_start, event_end] = interval;
+
+  // Loop over all bank sizes in all blocks
+  for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
+    auto const& [block_header, block_data] = blocks[i_block];
+    auto lhcb_type = block_header->bank_types()[0];
+    auto allen_type = bank_ids[lhcb_type];
+    auto& source_offsets = input_offsets[i_block];
+    uint run_number = 0;
+    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
+      for (size_t i = event_start; i < event_end; ++i) {
+        // First check for run changes in ODIN banks
+        if (split_by_run && lhcb_type == LHCb::RawBank::ODIN) {
+          // decode ODIN banks to check for run changes
+          auto odin_data = reinterpret_cast<unsigned int const*>(block_data.data() + source_offsets[i]);
+          unsigned const odin_run_number = odin_data[LHCb::ODIN::Data::RunNumber];
+          // if splitting by run, check all events have same run number
+          if (i == event_start) {
+            run_number = odin_run_number;
+          }
+          else if (odin_run_number != run_number) {
+            event_end = i;
+            break;
+          }
+        }
+        // Anticipate offset structure already here, i.e. don't assign to the first one
+        auto idx = i - event_start + 1;
+        auto& event_offsets = std::get<2>(slices[allen_type][slice_index]);
+
+        // Allen raw bank format has the sourceID followed by the raw bank data
+        event_offsets[idx] += sizeof(uint32_t) + block_header->bank_sizes()[i];
+      }
+    }
+  }
+
+  // Prefix sum over sizes per bank type per event to get the output
+  // "Allen" offsets per bank type per event
+  size_t n_frag = (event_end - event_start);
+  for (size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type) {
+    auto allen_type = bank_ids[lhcb_type];
+    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
+      auto& [slice, slice_size, event_offsets, offsets_size] = slices[allen_type][slice_index];
+      event_offsets[0] = 0;
+      auto preamble_words = 2 + banks_count[lhcb_type];
+      for (size_t i = 1; i <= (event_end - event_start) && i <= n_frag; ++i) {
+
+        // Allen raw bank format has the number of banks and the bank
+        // offsets in a preamble
+        event_offsets[i] += preamble_words * sizeof(uint32_t) + event_offsets[i - 1];
+
+        // Check for sufficient space
+        if (event_offsets[i] > slice_size) {
+          n_frag = i - 1;
+          break;
+        }
+      }
+    }
+  }
+
+  // Set offsets_size here to make sure it's consistent with the max
+  for (size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type) {
+    auto allen_type = bank_ids[lhcb_type];
+    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
+      auto& offsets_size = std::get<3>(slices[allen_type][slice_index]);
+      offsets_size = n_frag + 1;
+    }
+  }
+  return n_frag;
+}
+
+std::tuple<bool, bool, size_t> MEP::mep_offsets(
+  ::Slices& slices,
+  int const slice_index,
+  std::vector<int> const& bank_ids,
+  std::unordered_set<BankTypes> const& bank_types,
+  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+  EventIDs& event_ids,
+  MEP::MEP_header const* mep_header,
+  MEP::Blocks const& blocks,
+  std::tuple<size_t, size_t> const& interval,
+  bool split_by_run)
+{
+
+  auto [event_start, event_end] = interval;
+
+  unsigned char prev_type = 0;
+  size_t offset_index = 0;
+  uint run_number = 0;
+  for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
+    auto const& [block_header, block_data] = blocks[i_block];
+    auto lhcb_type = block_header->bank_types()[0];
+    auto allen_type = bank_ids[lhcb_type];
+    auto n_blocks = banks_count[lhcb_type];
+
+    // Decode ODIN banks
+    if (lhcb_type == LHCb::RawBank::ODIN) {
+      // decode ODIN bank to obtain run and event numbers
+      unsigned fragment_offset = 0;
+      for (unsigned i_event = 0; i_event < event_end; ++i_event) {
+        if (i_event >= event_start) {
+          auto const* odin_data = reinterpret_cast<unsigned int const*>(block_data.data() + fragment_offset);
+          auto const [odin_run_number, event_number] = decode_odin(odin_data);
+
+          // if splitting by run, check all events have same run number
+          if (i_event == event_start) {
+            run_number = odin_run_number;
+          }
+          else if (split_by_run && odin_run_number != run_number) {
+            event_end = i_event;
+            break;
+          }
+          event_ids.emplace_back(odin_run_number, event_number);
+        }
+        fragment_offset += block_header->bank_sizes()[i_event];
+      }
+    }
+
+    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
+      auto& [spans, data_size, event_offsets, offsets_size] = slices[allen_type][slice_index];
+
+      // Calculate block offset and size
+      size_t interval_offset = 0, interval_size = 0;
+      for (size_t i = 0; i < event_start; ++i) {
+        interval_offset += block_header->bank_sizes()[i];
+      }
+      for (size_t i = event_start; i < event_end; ++i) {
+        interval_size += block_header->bank_sizes()[i];
+      }
+
+      // Calculate offsets
+      if (lhcb_type != prev_type) {
+        event_offsets[0] = banks_count[lhcb_type];
+        event_offsets[1] = event_end - event_start;
+        event_offsets[2 + n_blocks] = 0;
+        offset_index = 0;
+        prev_type = lhcb_type;
+      }
+
+      // Store source ID
+      event_offsets[2 + offset_index] = mep_header->src_ids()[i_block];
+
+      // Initialize the first offsets using the block sizes,
+      if (offset_index < banks_count[lhcb_type] - 1) {
+        event_offsets[2 + n_blocks + offset_index + 1] = event_offsets[2 + n_blocks + offset_index] + interval_size;
+      }
+
+      // Fill fragment offsets
+      size_t oi = 0, idx = 0;
+      for (size_t i = event_start; i < event_end; ++i) {
+        idx = i - event_start + 1;
+        oi = 2 + n_blocks * (1 + idx) + offset_index;
+        event_offsets[oi] = event_offsets[oi - n_blocks] + block_header->bank_sizes()[i];
+      }
+      // Update offsets_size
+      offsets_size = oi;
+
+      // Store block span for this interval
+      spans.emplace_back(const_cast<char*>(block_data.data()) + interval_offset, interval_size);
+      data_size += interval_size;
+
+      ++offset_index;
+    }
+  }
+  return {true, false, event_end - event_start};
+}
+
+bool MEP::transpose_event(
+  ::Slices& slices,
+  int const slice_index,
+  std::vector<int> const& bank_ids,
+  std::unordered_set<BankTypes> const& bank_types,
+  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+  EventIDs& event_ids,
+  MEP::MEP_header const* mep_header,
+  MEP::Blocks const& blocks,
+  MEP::SourceOffsets const& input_offsets,
+  std::tuple<size_t, size_t> const& interval)
+{
+  auto [start_event, end_event] = interval;
+
+  // Loop over all bank data of this event
+  size_t bank_index = 1;
+  // L0Calo doesn't exist in the upgrade
+  LHCb::RawBank::BankType prev_type = LHCb::RawBank::L0Calo;
+
+  for (size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
+    auto const& [block_header, block_data] = blocks[i_block];
+    auto bank_type = static_cast<LHCb::RawBank::BankType>(block_header->bank_types()[0]);
+    auto& source_offsets = input_offsets[i_block];
+
+    // Check what to do with this bank
+    if (bank_type == LHCb::RawBank::ODIN) {
+      // decode ODIN bank to obtain run and event numbers
+      for (uint16_t i_event = start_event; i_event < end_event; ++i_event) {
+        auto odin_data = reinterpret_cast<unsigned int const*>(block_data.data() + source_offsets[i_event]);
+        event_ids.emplace_back(decode_odin(odin_data));
+      }
+    }
+
+    auto const allen_type = bank_ids[bank_type];
+    if (bank_type >= LHCb::RawBank::LastType || allen_type == -1 || !bank_types.count(BankTypes {allen_type})) {
+      prev_type = bank_type;
+    }
+    else {
+      if (bank_type != prev_type) {
+        bank_index = 1;
+        prev_type = bank_type;
+      }
+
+      auto allen_type = bank_ids[bank_type];
+      auto& slice = std::get<0>(slices[allen_type][slice_index])[0];
+      auto const& event_offsets = std::get<2>(slices[allen_type][slice_index]);
+
+      for (size_t i_event = start_event; i_event < end_event && i_event < block_header->n_banks; ++i_event) {
+        // Three things to write for a new set of banks:
+        // - number of banks/offsets
+        // - offsets to individual banks
+        // - bank data
+
+        auto preamble_words = 2 + banks_count[bank_type];
+
+        // Initialize point to write from offset of previous set
+        // All bank offsets are uit32_t so cast to that type
+        auto* banks_write = reinterpret_cast<uint32_t*>(slice.data() + event_offsets[i_event - start_event]);
+
+        // Where to write the offsets
+        auto* banks_offsets_write = banks_write + 1;
+
+        if (bank_index == 1) {
+          // Write the number of banks
+          banks_write[0] = banks_count[bank_type];
+          banks_offsets_write[0] = 0;
+        }
+
+        // get offset for this bank and store offset for next bank
+        auto offset = banks_offsets_write[bank_index - 1];
+        auto frag_size = block_header->bank_sizes()[i_event];
+        banks_offsets_write[bank_index] = offset + frag_size + sizeof(uint32_t);
+
+        // Where to write the bank data itself
+        banks_write += preamble_words;
+
+        // Write sourceID; offset in 32bit words
+        auto word_offset = offset / sizeof(uint32_t);
+        banks_write[word_offset] = mep_header->src_ids()[i_block];
+
+        // Write bank data
+        std::memcpy(banks_write + word_offset + 1, block_data.data() + source_offsets[i_event], frag_size);
+      }
+
+      ++bank_index;
+    }
+  }
+  return true;
+}
+
+std::tuple<bool, bool, size_t> MEP::transpose_events(
+  ::Slices& slices,
+  int const slice_index,
+  std::vector<int> const& bank_ids,
+  std::unordered_set<BankTypes> const& bank_types,
+  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+  EventIDs& event_ids,
+  MEP::MEP_header const* mep_header,
+  MEP::Blocks const& blocks,
+  MEP::SourceOffsets const& source_offsets,
+  std::tuple<size_t, size_t> const& interval,
+  bool split_by_run)
+{
+  auto [event_start, event_end] = interval;
+
+  bool success = true;
+
+  auto to_transpose = allen_offsets(
+    slices, slice_index, bank_ids, bank_types, banks_count, blocks, source_offsets, interval, split_by_run);
+
+  transpose_event(
+    slices,
+    slice_index,
+    bank_ids,
+    bank_types,
+    banks_count,
+    event_ids,
+    mep_header,
+    blocks,
+    source_offsets,
+    {event_start, event_start + to_transpose});
+
+  return {success, to_transpose != (event_end - event_start), to_transpose};
+}
diff --git a/AllenOnline/src/TransposeMEP.h b/AllenOnline/src/TransposeMEP.h
new file mode 100644
index 000000000..079020c23
--- /dev/null
+++ b/AllenOnline/src/TransposeMEP.h
@@ -0,0 +1,137 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#pragma once
+
+#include <thread>
+#include <vector>
+#include <array>
+#include <deque>
+#include <mutex>
+#include <atomic>
+#include <chrono>
+#include <algorithm>
+#include <numeric>
+#include <condition_variable>
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+
+#include <Allen/Common.h>
+#include <Allen/AllenUnits.h>
+#include <Allen/TransposeTypes.h>
+#include <Event/RawBank.h>
+
+#include <EventBuilding/MFP_tools.hpp>
+#include <EventBuilding/MEP_tools.hpp>
+
+namespace {
+  using namespace Allen::Units;
+} // namespace
+
+namespace MEP {
+
+  using SourceOffsets = std::vector<std::vector<uint32_t>>;
+  using Blocks = std::vector<std::tuple<MFP::MFP_header const*, gsl::span<char const>>>;
+
+  struct Slice {
+    MEP::MEP_header const* mep_header = nullptr;
+    unsigned packing_factor = 0u;
+    gsl::span<char const> mep_data;
+    Blocks blocks;
+    SourceOffsets offsets;
+    size_t slice_size = 0u;
+  };
+  using Slices = std::vector<MEP::Slice>;
+
+  /**
+   * @brief      Fill the array the contains the number of banks per type
+   *
+   * @details    detailed description
+   *
+   * @param      MEP::MEP_header for a MEP
+   * @param      span of the block data in the MEP
+   *
+   * @return     (success, number of banks per bank type; 0 if the bank is not needed)
+   */
+  std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>> fill_counts(
+    MEP::MEP_header const* header,
+    gsl::span<char const> const& data);
+
+  void find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor, gsl::span<char const> const& buffer_span, Blocks& blocks);
+
+  void fragment_offsets(Blocks const& blocks, std::vector<std::vector<uint32_t>>& offsets);
+
+  size_t allen_offsets(
+    ::Slices& slices,
+    int const slice_index,
+    std::vector<int> const& bank_ids,
+    std::unordered_set<BankTypes> const& bank_types,
+    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    Blocks const& blocks,
+    MEP::SourceOffsets const& input_offsets,
+    std::tuple<size_t, size_t> const& interval,
+    bool split_by_run = false);
+
+  std::tuple<bool, bool, size_t> mep_offsets(
+    ::Slices& slices,
+    int const slice_index,
+    std::vector<int> const& bank_ids,
+    std::unordered_set<BankTypes> const& bank_types,
+    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    EventIDs& event_ids,
+    MEP::MEP_header const* mep_header,
+    MEP::Blocks const& blocks,
+    std::tuple<size_t, size_t> const& interval,
+    bool split_by_run = false);
+
+  /**
+   * @brief      Transpose events to Allen layout
+   *
+   * @param      slices to fill with transposed banks, slices are addressed by bank type
+   * @param      index of bank slices
+   * @param      number of banks per event
+   * @param      event ids of banks in this slice
+   * @param      start of bank data for this event
+   *
+   * @return     tuple of: (success, slice is full)
+   */
+  bool transpose_event(
+    ::Slices& slices,
+    int const slice_index,
+    std::vector<int> const& bank_ids,
+    std::unordered_set<BankTypes> const& bank_types,
+    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    EventIDs& event_ids,
+    MEP::MEP_header const* mep_header,
+    Blocks const& blocks,
+    SourceOffsets const& input_offsets,
+    std::tuple<size_t, size_t> const& interval);
+
+  /**
+   * @brief      Transpose MEP to Allen layout
+   *
+   * @param      slices to fill with transposed banks, slices are addressed by bank type
+   * @param      index of bank slices
+   * @param      number of banks per event
+   * @param      event ids of banks in this slice
+   * @param      start of bank data for this event
+   *
+   * @return     tuple of: (success, slice is full)
+   */
+  std::tuple<bool, bool, size_t> transpose_events(
+    ::Slices& slices,
+    int const slice_index,
+    std::vector<int> const& bank_ids,
+    std::unordered_set<BankTypes> const& bank_types,
+    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    EventIDs& event_ids,
+    MEP::MEP_header const* mep_header,
+    Blocks const& blocks,
+    SourceOffsets const& source_offsets,
+    std::tuple<size_t, size_t> const& interval,
+    bool split_by_run = false);
+
+} // namespace MEP
diff --git a/AllenOnline/src/WriteMDF.h b/AllenOnline/src/WriteMDF.h
new file mode 100644
index 000000000..ebf5d08db
--- /dev/null
+++ b/AllenOnline/src/WriteMDF.h
@@ -0,0 +1,25 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <cstring>
+
+#include <gsl/gsl>
+
+#include <Event/RawBank.h>
+
+inline size_t add_raw_bank(
+  unsigned char const type,
+  unsigned char const version,
+  short const sourceID,
+  gsl::span<char const> fragment,
+  char* buffer)
+{
+  auto* bank = reinterpret_cast<LHCb::RawBank*>(buffer);
+  bank->setMagic();
+  bank->setSize(fragment.size());
+  bank->setType(static_cast<LHCb::RawBank::BankType>(type));
+  bank->setVersion(version);
+  bank->setSourceID(sourceID);
+  std::memcpy(bank->begin<char>(), fragment.data(), fragment.size());
+  return bank->size() + bank->hdrSize();
+}
-- 
GitLab


From 720f23031a39b46d7d2b760be174db42ec80ba8a Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 27 May 2021 14:36:51 +0200
Subject: [PATCH 003/120] Fix MEPProvider build with MPI and add AllenMPISend
 application

---
 AllenOnline/CMakeLists.txt                    |  49 ++-
 AllenOnline/application/MPISend.cpp           | 238 +++++++++++++++
 .../application/bench_mep_provider.cpp        |  66 ++++
 AllenOnline/application/mdf_to_mep.cpp        | 287 ++++++++++++++++++
 AllenOnline/include/MPIConfig.h               |  56 ++++
 AllenOnline/{src => include}/ReadMEP.h        |   0
 AllenOnline/src/AllenApplication.cpp          |   2 +-
 AllenOnline/src/AllenApplication.h            |   8 +-
 AllenOnline/src/MEPProvider.cpp               |  32 +-
 AllenOnline/src/MEPProvider.h                 |   7 +-
 10 files changed, 709 insertions(+), 36 deletions(-)
 create mode 100644 AllenOnline/application/MPISend.cpp
 create mode 100644 AllenOnline/application/bench_mep_provider.cpp
 create mode 100644 AllenOnline/application/mdf_to_mep.cpp
 create mode 100644 AllenOnline/include/MPIConfig.h
 rename AllenOnline/{src => include}/ReadMEP.h (100%)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index e474508a8..1e7b29d29 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -14,37 +14,58 @@ gaudi_depends_on_subdirs(Online/OnlineKernel
 
 find_package(cppgsl REQUIRED)
 find_package(ZMQ REQUIRED)
-find_package(Boost REQUIRED)
+find_package(Boost REQUIRED COMPONENTS program_options)
 find_package(ROOT REQUIRED)
+find_package(CUDAToolkit REQUIRED)
 
 include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
   ${ZMQ_INCLUDE_DIRS}
   ${CPPGSL_INCLUDE_DIR}
   ${ROOT_INCLUDE_DIRS})
 
-# find_package(MPI REQUIRED)
-# include_directories(SYSTEM ${MPI_CXX_INCLUDE_PATH})
-# add_compile_definitions(HAVE_MPI)
+include_directories(include)
 
-include_directories(${CMAKE_SOURCE_DIR}/main/include)
+gaudi_add_library(AllenOnline
+                 src/ReadMEP.cpp
+                 src/TransposeMEP.cpp
+                 NO_PUBLIC_HEADERS
+                 INCLUDE_DIRS GaudiKernel cppgsl Online/OnlineKernel Online/EventBuilding
+                 LINK_LIBRARIES GaudiKernel Allen::AllenLib EventBuildingLib)
 
-gaudi_add_module(AllenOnline
-                 src/*.cpp
+gaudi_add_module(AllenOnlineComp
+                 src/AllenApplication.cpp
+                 src/AllenConfiguration.cpp
+                 src/MEPProvider.cpp
                  INCLUDE_DIRS OnlineBase ROOT Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
                  LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQLib RPC Parsers EventBuildingLib MDFLib
-                                OnlineKernel Allen::AllenLib Allen::BinaryDumpers DAQEventLib DAQKernelLib GaudiAlgLib
-                                PrKernel VPDetLib UTDetLib UTKernelLib
+                                OnlineKernel Allen::AllenLib Allen::BinaryDumpers Allen::AllenRuntime DAQEventLib
+                                DAQKernelLib GaudiAlgLib PrKernel VPDetLib UTDetLib UTKernelLib AllenOnline
                                 FTDetLib -lrt)
 
 # MPI
 find_package(MPI QUIET COMPONENTS C CXX)
 if (MPI_FOUND)
-  message(STATUS "Found MPI ${MPI_VERSION}: ${MPI_CXX_COMPILER}")
+  message(STATUS "Found MPI ${MPI_CXX_VERSION}: ${MPI_CXX_COMPILER}")
   #hwloc is also needed
   find_package(PkgConfig REQUIRED)
-  pkg_check_modules(HWLOC REQUIRED IMPORTED_TARGET hwloc)
+  pkg_check_modules(hwloc REQUIRED IMPORTED_TARGET hwloc)
 
-  target_link_libraries(AllenOnline PUBLIC MPI::MPI_CXX PkgConfig::HWLOC open-pal)
-  target_include_directories(AllenOnline SYSTEM PUBLIC ${MPI_CXX_INCLUDE_DIRS})
-  target_compile_definitions(AllenOnline PRIVATE HAVE_MPI)
+  # open-pal library is also needed...
+  set(MPI_LIBDIRS)
+  foreach(mpi_lib ${MPI_CXX_LIBRARIES})
+    get_filename_component(_lib_dir ${mpi_lib} DIRECTORY)
+    list(APPEND MPI_LIBDIRS ${_lib_dir})
+  endforeach()
+  find_library(open_pal_lib open-pal PATHS ${MPI_LIBDIRS} REQUIRED)
+
+  target_link_libraries(AllenOnlineComp MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
+  target_include_directories(AllenOnlineComp PRIVATE SYSTEM ${MPI_CXX_INCLUDE_DIRS})
+  target_compile_definitions(AllenOnlineComp PRIVATE HAVE_MPI)
+
+
+  gaudi_add_executable(AllenMPISend
+                       application/MPISend.cpp
+                       INCLUDE_DIRS GaudiKernel ${MPI_CXX_INCLUDE_DIRS}
+                       LINK_LIBRARIES AllenOnline MDFLib Boost::program_options MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
+  target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
 endif()
diff --git a/AllenOnline/application/MPISend.cpp b/AllenOnline/application/MPISend.cpp
new file mode 100644
index 000000000..e22ef15f2
--- /dev/null
+++ b/AllenOnline/application/MPISend.cpp
@@ -0,0 +1,238 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <map>
+#include <string>
+#include <iostream>
+#include <vector>
+
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <unistd.h>
+
+#include <boost/program_options.hpp>
+#include <boost/algorithm/string.hpp>
+
+#include <ReadMEP.h>
+#include <MPIConfig.h>
+
+namespace MPI {
+  int rank;
+
+  std::string rank_str()
+  {
+    if (rank == receiver) {
+      return "MPI::Receiver: ";
+    }
+    else {
+      return "MPI::Sender: ";
+    }
+  }
+} // namespace MPI
+
+namespace {
+  namespace po = boost::program_options;
+  namespace ba = boost::algorithm;
+}
+
+int main(int argc, char* argv[]) {
+
+  std::string mep_input;
+  size_t window_size = 4;
+  bool non_stop = false;
+  size_t number_of_meps = 0;
+
+  po::options_description desc("Allowed options");
+  desc.add_options()
+    ("help", "produce help message")
+    ("mep", po::value<std::string>(&mep_input),
+     "mep files as comma-separated list")
+    ("mpi-window-size", po::value<size_t>(&window_size)->default_value(4),
+     "MPI sliding window size")
+    ("non-stop", po::value<bool>(&non_stop)->default_value(false), "input file")
+    ("n-meps,n", po::value<size_t>(&number_of_meps)->default_value(0), "#MEPs");
+
+  po::positional_options_description p;
+  p.add("mep", 1);
+
+  po::variables_map vm;
+  po::store(po::command_line_parser(argc, argv).options(desc).positional(p).run(), vm);
+  po::notify(vm);
+
+  if (vm.count("help")) {
+    std::cout << desc << "\n";
+    return 1;
+  }
+
+  // The sender is in charge of reading all MDF files and passing
+  // them to the receiver.
+
+  if (!vm.count("mep")) {
+    std::cout << MPI::rank_str() << "Required argument --mep not supplied. Exiting application.\n";
+    return -1;
+  }
+
+  std::vector<std::string> connections;
+  ba::split(connections, mep_input, ba::is_any_of(","));
+
+  // Create requests of appropiate size
+  std::vector<MPI_Request> requests(window_size);
+
+  // Read all files in connections
+  std::vector<std::tuple<MEP::MEP_header const*, gsl::span<char>>> meps;
+
+  std::cout << MPI::rank_str() << "Reading "
+            << (number_of_meps != 0 ? std::to_string(number_of_meps) : std::string {"all"}) << " meps from files\n";
+
+  std::vector<char> data;
+  gsl::span<char const> mep_span;
+  size_t n_meps_read = 0;
+
+  std::optional<unsigned> packing_factor;
+
+  for (const auto& connection : connections) {
+    bool eof = false, success = true;
+    MEP::MEP_header const* mep_header = nullptr;
+    unsigned pf = 0;
+
+    auto input = LHCb::StreamDescriptor::connect(connection);
+    if (input.ioDesc != 0) {
+      std::cout << "Opened " << connection << "\n";
+    }
+
+    while (success && !eof) {
+      std::cout << "." << std::flush;
+
+      std::tie(eof, success, mep_header, pf, mep_span) = MEP::read_mep(input, data);
+      if (!packing_factor) {
+        packing_factor = pf;
+      }
+      else if (*packing_factor != pf) {
+        std::cout << "Got MEP with different packing factor: " << pf
+                  << " instead of: " << *packing_factor << "\n";
+        return 1;
+      }
+
+      if (!eof && success) {
+        char* contents = nullptr;
+        MPI_Alloc_mem(mep_span.size(), MPI_INFO_NULL, &contents);
+
+        // Populate contents with stream buf
+        std::copy_n(mep_span.data(), mep_span.size(), contents);
+        ++n_meps_read;
+
+        meps.emplace_back(mep_header, gsl::span<char> {contents, mep_span.size()});
+      }
+      if (n_meps_read >= number_of_meps && number_of_meps != 0) {
+        input.close();
+        goto send;
+      }
+    }
+    input.close();
+  }
+
+send:
+
+  if (meps.empty() || !packing_factor) {
+    std::cout << "Failed to read MEPs from file\n";
+    return 1;
+  }
+
+  // MPI initialization
+  MPI_Init(&argc, &argv);
+
+  // Communication size
+  int comm_size;
+  MPI_Comm_size(MPI_COMM_WORLD, &comm_size);
+  if (comm_size > MPI::comm_size) {
+    std::cout << "This program requires at most " << MPI::comm_size << " processes.\n";
+    return -1;
+  }
+
+  // MPI: Who am I?
+  MPI_Comm_rank(MPI_COMM_WORLD, &MPI::rank);
+
+  auto const& first_header = std::get<0>(*meps.begin());
+
+  size_t pf = *packing_factor;
+  std::cout << "\n"
+            << MPI::rank_str() << "MEP header: " << first_header->n_MFPs << ", " << pf << ", "
+            << first_header->bytes() << "\n";
+  MPI_Send(&pf, 1, MPI_SIZE_T, MPI::receiver, MPI::message::packing_factor, MPI_COMM_WORLD);
+
+  MPI_Send(&n_meps_read, 1, MPI_SIZE_T, MPI::receiver, MPI::message::number_of_meps, MPI_COMM_WORLD);
+
+  // Test: Send all the files
+  size_t current_mep = 0;
+  while (non_stop || current_mep < meps.size()) {
+
+    // Get event data
+    auto const& [mep_header, mep_span] = meps[current_mep];
+    const char* current_event_start = mep_span.data();
+    const size_t current_event_size = mep_span.size();
+
+    // Notify the event size
+    MPI_Send(&current_event_size, 1, MPI_SIZE_T, MPI::receiver, MPI::message::event_size, MPI_COMM_WORLD);
+
+    // Number of full-size (MPI::mdf_chunk_size) messages
+    size_t n_messages = current_event_size / MPI::mdf_chunk_size;
+    // Size of the last message (if the MFP size is not a multiple of MPI::mdf_chunk_size)
+    size_t rest = current_event_size - n_messages * MPI::mdf_chunk_size;
+    // Number of parallel sends
+    size_t n_sends = n_messages > window_size ? window_size : n_messages;
+
+    // Initial parallel sends
+    for (size_t k = 0; k < n_sends; k++) {
+      const char* message = current_event_start + k * MPI::mdf_chunk_size;
+      MPI_Isend(
+        message,
+        MPI::mdf_chunk_size,
+        MPI_BYTE,
+        MPI::receiver,
+        MPI::message::event_send_tag_start + k,
+        MPI_COMM_WORLD,
+        &requests[k]);
+    }
+    // Sliding window sends
+    for (size_t k = n_sends; k < n_messages; k++) {
+      int r;
+      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      const char* message = current_event_start + k * MPI::mdf_chunk_size;
+      MPI_Isend(
+        message,
+        MPI::mdf_chunk_size,
+        MPI_BYTE,
+        MPI::receiver,
+        MPI::message::event_send_tag_start + k,
+        MPI_COMM_WORLD,
+        &requests[r]);
+    }
+    // Last send (if necessary)
+    if (rest) {
+      int r;
+      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      const char* message = current_event_start + n_messages * MPI::mdf_chunk_size;
+      MPI_Isend(
+        message,
+        rest,
+        MPI_BYTE,
+        MPI::receiver,
+        MPI::message::event_send_tag_start + n_messages,
+        MPI_COMM_WORLD,
+        &requests[r]);
+    }
+    // Wait until all chunks have been sent
+    MPI_Waitall(n_sends, requests.data(), MPI_STATUSES_IGNORE);
+
+    if (non_stop) {
+      current_mep = (current_mep + 1) % meps.size();
+    }
+    else {
+      ++current_mep;
+    }
+  }
+
+  MPI_Finalize();
+  return 0;
+}
diff --git a/AllenOnline/application/bench_mep_provider.cpp b/AllenOnline/application/bench_mep_provider.cpp
new file mode 100644
index 000000000..1bf0d7fbc
--- /dev/null
+++ b/AllenOnline/application/bench_mep_provider.cpp
@@ -0,0 +1,66 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <cstring>
+#include <iostream>
+#include <fstream>
+#include <string>
+#include <iomanip>
+#include <unordered_set>
+#include <map>
+
+#include <Event/RawBank.h>
+#include <Timer.h>
+#include <MEPProvider.h>
+
+using namespace std;
+
+int main(int argc, char* argv[])
+{
+  if (argc <= 1) {
+    cout << "usage: bench_provider <file.mep> <file.mep> <file.mep> ..." << endl;
+    return -1;
+  }
+
+  string filename = {argv[1]};
+  size_t n_slices = 10;
+  size_t events_per_slice = 1000;
+  double n_filled = 0.;
+
+  vector<string> files(argc - 1);
+  for (int i = 0; i < argc - 1; ++i) {
+    files[i] = argv[i + 1];
+  }
+
+  logger::setVerbosity(4);
+
+  Timer t;
+
+  MEPProviderConfig config {false,         // verify MEP checksums
+                            10,            // number of read buffers
+                            2,             // number of transpose threads
+                            4,             // MPI sliding window size
+                            false,         // Receive from MPI or read files
+                            false,         // Run the application non-stop
+                            true,          // Transpose MEP
+                            false,         // Split by run number
+                            {{"mem", 0}}}; // mapping of receiver to its numa node
+
+  MEPProvider<BankTypes::VP, BankTypes::UT, BankTypes::FT, BankTypes::MUON> mep {
+    n_slices, events_per_slice, {}, files, config};
+
+  chrono::milliseconds sleep_interval {10};
+
+  bool good = true, done = false, timed_out = false;
+  size_t filled = 0, slice = 0;
+  uint runno = 0;
+  while (good || filled != 0) {
+    std::tie(good, done, timed_out, slice, filled, runno) = mep.get_slice();
+    n_filled += filled;
+    this_thread::sleep_for(sleep_interval);
+    mep.slice_free(slice);
+  }
+
+  t.stop();
+  cout << "Filled " << n_filled / t.get() << " events/s\n";
+}
diff --git a/AllenOnline/application/mdf_to_mep.cpp b/AllenOnline/application/mdf_to_mep.cpp
new file mode 100644
index 000000000..f42094fb4
--- /dev/null
+++ b/AllenOnline/application/mdf_to_mep.cpp
@@ -0,0 +1,287 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <cstring>
+#include <iostream>
+#include <fstream>
+#include <string>
+#include <iomanip>
+#include <unordered_set>
+#include <numeric>
+#include <map>
+#include <cassert>
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+
+#include <Event/RawBank.h>
+#include <read_mdf.hpp>
+#include <eb_header.hpp>
+#include <Transpose.h>
+#include <Common.h>
+#include <BankTypes.h>
+
+using namespace std;
+
+namespace detail {
+
+  template<typename T>
+  std::ostream& write(std::ostream& os, const T& t)
+  {
+    // if you would like to know why there is a check for trivially copyable,
+    // please read the 'notes' section of https://en.cppreference.com/w/cpp/types/is_trivially_copyable
+    if constexpr (gsl::details::is_span<T>::value) {
+      return os.write(reinterpret_cast<const char*>(t.data()), t.size_bytes());
+    }
+    else if constexpr (std::is_trivially_copyable_v<T> && !gsl::details::is_span<T>::value) {
+      return os.write(reinterpret_cast<const char*>(&t), sizeof(T));
+    }
+    else {
+      static_assert(std::is_trivially_copyable_v<typename T::value_type>);
+      return write(os, as_bytes(gsl::make_span(t)));
+    }
+  }
+
+} // namespace detail
+
+class FileWriter {
+  std::ofstream m_f;
+
+public:
+  FileWriter(const std::string& name) : m_f {name, std::ios::out | std::ios::binary} {}
+
+  bool is_open() { return m_f.is_open(); }
+
+  template<typename... Args>
+  FileWriter& write(Args&&... args)
+  {
+    (detail::write(m_f, std::forward<Args>(args)), ...);
+    return *this;
+  }
+};
+
+int main(int argc, char* argv[])
+{
+
+  if (argc < 4) {
+    cout << "usage: mdf_to_mep output_file #MEPs packing_factor input.mdf ...\n";
+    return -1;
+  }
+
+  string output_file {argv[1]};
+  size_t n_meps = std::atol(argv[2]);
+  uint16_t packing_factor = std::atoi(argv[3]);
+
+  vector<string> input_files(argc - 4);
+  for (int i = 0; i < argc - 4; ++i) {
+    input_files[i] = argv[i + 4];
+  }
+
+  vector<char> buffer(1024 * 1024, '\0');
+  vector<char> decompression_buffer(1024 * 1024, '\0');
+
+  LHCb::MDFHeader mdf_header;
+  bool error = false;
+  bool eof = false;
+  gsl::span<char> bank_span;
+
+  bool sizes_known = false;
+  bool count_success = false;
+  std::array<unsigned int, LHCb::NBankTypes> banks_count;
+
+  size_t n_read = 0;
+  size_t n_written = 0;
+  uint64_t event_id = 0;
+
+  std::vector<std::tuple<EB::BlockHeader, size_t, vector<char>>> blocks;
+  EB::Header mep_header;
+
+  // offsets to fragments of the detector types
+  std::array<size_t, LHCb::NBankTypes> block_offsets {0};
+
+  // Header version 3
+  auto hdr_size = LHCb::MDFHeader::sizeOf(3);
+  std::vector<char> header_buffer(hdr_size, '\0');
+  auto* header = reinterpret_cast<LHCb::MDFHeader*>(header_buffer.data());
+  header->setHeaderVersion(3);
+  header->setDataType(LHCb::MDFHeader::BODY_TYPE_MEP);
+  header->setSubheaderLength(hdr_size - sizeof(LHCb::MDFHeader));
+
+  FileWriter writer {output_file};
+  if (!writer.is_open()) {
+    cerr << "Failed to open output file: " << strerror(errno) << "\n";
+    return -1;
+  }
+
+  auto write_fragments = [&writer, &blocks, &n_written, &mep_header, hdr_size, packing_factor, header] {
+    header->setSize(
+      mep_header.header_size(blocks.size()) +
+      std::accumulate(blocks.begin(), blocks.end(), 0, [packing_factor](size_t s, const auto& entry) {
+        auto& [block_header, n_filled, data] = entry;
+        return s + block_header.header_size(packing_factor) + block_header.block_size;
+      }));
+    writer.write(gsl::span {reinterpret_cast<char const*>(header), hdr_size});
+
+    size_t block_offset = 0;
+    for (size_t ib = 0; ib < blocks.size(); ++ib) {
+      mep_header.offsets[ib] = block_offset;
+      auto const& block = std::get<0>(blocks[ib]);
+      block_offset += block.header_size(block.n_frag) + block.block_size;
+    }
+    mep_header.mep_size = block_offset;
+    mep_header.packing_factor = packing_factor;
+
+    writer.write(
+      mep_header.n_blocks,
+      mep_header.packing_factor,
+      mep_header.reserved,
+      mep_header.mep_size,
+      mep_header.source_ids,
+      mep_header.versions,
+      mep_header.offsets);
+
+    for (auto& [block_header, n_filled, data] : blocks) {
+      assert(std::accumulate(block_header.sizes.begin(), block_header.sizes.end(), 0u) == block_header.block_size);
+      writer.write(
+        block_header.event_id,
+        block_header.n_frag,
+        block_header.reserved,
+        block_header.block_size,
+        block_header.types,
+        block_header.sizes);
+      writer.write(gsl::span {data.data(), block_header.block_size});
+
+      // Reset the fragments
+      block_header.block_size = 0;
+      n_filled = 0;
+    }
+    ++n_written;
+  };
+
+  for (auto const& file : input_files) {
+    auto input = MDF::open(file.c_str(), O_RDONLY);
+    if (input.good) {
+      cout << "Opened " << file << "\n";
+    }
+    else {
+      cerr << "Failed to open " << file << " " << strerror(errno) << "\n";
+      error = true;
+      break;
+    }
+    while (!eof && n_written < n_meps) {
+      std::tie(eof, error, bank_span) = MDF::read_event(input, mdf_header, buffer, decompression_buffer, false);
+      if (eof) {
+        eof = false;
+        break;
+      }
+      else if (error) {
+        cerr << "Failed to read event\n";
+        return -1;
+      }
+      else {
+        ++n_read;
+      }
+
+      if (!sizes_known) {
+        // Count the number of banks of each type and the start of the
+        // source ID range
+        std::tie(count_success, banks_count) = fill_counts(bank_span);
+        // Skip DAQ bank
+        uint16_t n_blocks =
+          std::accumulate(banks_count.begin(), banks_count.end(), 0) - banks_count[LHCb::RawBank::DAQ];
+        size_t offset = 0, i = 0;
+        for (i = 0; i < banks_count.size(); ++i) {
+          if (i != to_integral(LHCb::RawBank::DAQ)) {
+            block_offsets[i] = offset;
+            offset += banks_count[i];
+          }
+        }
+        blocks.resize(n_blocks);
+        for (auto& block : blocks) {
+          std::get<2>(block).resize(packing_factor * average_event_size * kB / n_blocks);
+        }
+
+        mep_header = EB::Header {packing_factor, n_blocks};
+        sizes_known = true;
+      }
+
+      // Put the banks in the event-local buffers
+      char const* bank = bank_span.data();
+      char const* end = bank + bank_span.size();
+      size_t source_offset = 0;
+      auto prev_type = LHCb::RawBank::L0Calo;
+      while (bank < end) {
+        const auto* b = reinterpret_cast<const LHCb::RawBank*>(bank);
+        if (b->magic() != LHCb::RawBank::MagicPattern) {
+          cout << "magic pattern failed: " << std::hex << b->magic() << std::dec << endl;
+          return -1;
+        }
+
+        // Skip the DAQ bank, it's created on read from the MDF header
+        if (b->type() < LHCb::RawBank::LastType && b->type() != LHCb::RawBank::DAQ) {
+          if (b->type() != prev_type) {
+            source_offset = 0;
+            prev_type = b->type();
+          }
+          else {
+            ++source_offset;
+          }
+          auto block_index = block_offsets[b->type()] + source_offset;
+          auto& [block_header, n_filled, data] = blocks[block_index];
+
+          if (n_filled == 0) {
+            mep_header.source_ids[block_index] = b->sourceID();
+            mep_header.versions[block_index] = b->version();
+            block_header = EB::BlockHeader {event_id, packing_factor};
+          }
+          else if (mep_header.source_ids[block_index] != b->sourceID()) {
+            cout << "Error: banks not ordered in the same way: " << mep_header.source_ids[block_index] << " "
+                 << b->sourceID() << "\n";
+            return -1;
+          }
+
+          // NOTE: All banks are truncated to 32 bit values. This
+          // doesn't seem to make a difference except for the UT,
+          // where the size is larger than the number of words o.O
+          auto n_words = b->size() / sizeof(uint32_t);
+          auto word_size = n_words * sizeof(uint32_t);
+          block_header.types[n_filled] = b->type();
+          block_header.sizes[n_filled] = n_words * sizeof(uint32_t);
+
+          // Resize on demand
+          if (block_header.block_size + word_size >= data.size()) {
+            data.resize(std::max(static_cast<size_t>(1.5 * data.size()), data.size() + word_size));
+          }
+
+          // Copy bank data
+          ::memcpy(&data[0] + block_header.block_size, b->data(), word_size);
+          block_header.block_size += word_size;
+
+          ++n_filled;
+        }
+        else if (b->type() != LHCb::RawBank::DAQ) {
+          cout << "unknown bank type: " << b->type() << endl;
+        }
+
+        // Move to next raw bank
+        bank += b->totalSize();
+      }
+
+      if (n_read % packing_factor == 0 && n_read != 0) {
+        write_fragments();
+        event_id += packing_factor;
+      }
+    }
+
+    input.close();
+    if (n_written >= n_meps) break;
+  }
+
+  if (!error) {
+    cout << "Wrote " << n_written << " MEPs with " << (n_read % packing_factor) << " events left over.\n";
+  }
+
+  return error ? -1 : 0;
+}
diff --git a/AllenOnline/include/MPIConfig.h b/AllenOnline/include/MPIConfig.h
new file mode 100644
index 000000000..42de41445
--- /dev/null
+++ b/AllenOnline/include/MPIConfig.h
@@ -0,0 +1,56 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#pragma once
+
+#include <mpi.h>
+#include <limits.h>
+
+// Determine size of size_t for MPI type
+#if SIZE_MAX == UCHAR_MAX
+#define MPI_SIZE_T MPI_UNSIGNED_CHAR
+#elif SIZE_MAX == USHRT_MAX
+#define MPI_SIZE_T MPI_UNSIGNED_SHORT
+#elif SIZE_MAX == UINT_MAX
+#define MPI_SIZE_T MPI_UNSIGNED
+#elif SIZE_MAX == ULONG_MAX
+#define MPI_SIZE_T MPI_UNSIGNED_LONG
+#elif SIZE_MAX == ULLONG_MAX
+#define MPI_SIZE_T MPI_UNSIGNED_LONG_LONG
+#else
+#error "size_t size could not be determined."
+#endif
+
+namespace MPI {
+  // MPI ranks of sender and receiver
+  constexpr int receiver = 0;
+
+  // Required MPI communication size (number of ranks)
+  constexpr int comm_size = 3;
+
+  // Chunk size of MDF events
+  // Note: With MEPs, this in principle may not be needed
+  // ie. 1 MiB
+  constexpr int mdf_chunk_size = 1024 * 1024;
+
+  // Rank of current process
+  extern int rank;
+
+  namespace message {
+    // Message tags
+    constexpr int test = 1;
+    constexpr int packing_factor = 2;
+    constexpr int number_of_meps = 3;
+    constexpr int max_file_size = 4;
+    constexpr int event_size = 5;
+    constexpr int window_size = 6;
+
+    // Event sends will start with tag start, and cycle every modulo
+    // ie:
+    // const auto tag = event_send_tag_start + (i % event_send_tag_modulo);
+    constexpr int event_send_tag_start = 100;
+    // constexpr int event_send_tag_modulo = 1024;
+  } // namespace message
+
+  std::string rank_str();
+} // namespace MPI
diff --git a/AllenOnline/src/ReadMEP.h b/AllenOnline/include/ReadMEP.h
similarity index 100%
rename from AllenOnline/src/ReadMEP.h
rename to AllenOnline/include/ReadMEP.h
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 6d12baa93..2090f0647 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -28,7 +28,7 @@
 #include <Allen/Allen.h>
 
 #ifdef HAVE_MPI
-#include <Allen/MPIConfig.h>
+#include <MPIConfig.h>
 #endif
 
 #include "AllenConfiguration.h"
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 83d6b0386..d294298eb 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -93,7 +93,9 @@ private:
   std::thread m_allenThread;
   std::optional<zmq::socket_t> m_allenControl;
 
-  // char** m_mpiArgv = nullptr;
-  // int m_mpiArgc = 1;
-  // int m_rank = -1;
+#ifdef HAVE_MPI
+  char** m_mpiArgv = nullptr;
+  int m_mpiArgc = 1;
+  int m_rank = -1;
+#endif
 };
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 9b8e0f07c..d4a74d368 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -7,6 +7,7 @@
 #include <Allen/BankTypes.h>
 #include <Allen/Timer.h>
 #include <Allen/SliceUtils.h>
+#include <Backend/BackendCommon.h>
 
 #include <MDF/StreamDescriptor.h>
 #include <MDF/MDFHeader.h>
@@ -384,6 +385,7 @@ void MEPProvider::init_mpi()
     // Only bind explicitly if there are multiple receivers,
     // otherwise assume a memory allocation policy is in effect
     if (m_domains.size() > 1) {
+      auto numa_node = i % m_config.n_receivers();
       auto const& numa_obj = numa_objs[numa_node];
       auto s = hwloc_set_area_membind(
         m_topology, contents, n_bytes, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
@@ -393,14 +395,14 @@ void MEPProvider::init_mpi()
       }
     }
 
-    cudaCheck(cudaHostRegister(contents, n_bytes, cudaHostRegisterDefault));
+    Allen::host_register(contents, n_bytes, Allen::hostRegisterDefault);
     m_net_slices.emplace_back(
-      nullptr,
-      0u,
-      gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
-      MEP::Blocks {},
-      MEP::SourceOffsets {},
-      n_bytes);
+      MEP::Slice{nullptr,
+                 0u,
+                 gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
+                 MEP::Blocks {},
+                 MEP::SourceOffsets {},
+                 n_bytes});
     m_mpi_buffers.emplace_back(contents);
   }
   #else
@@ -749,16 +751,16 @@ void MEPProvider::mpi_read()
     MPI_Recv(&mep_size, 1, MPI_SIZE_T, sender_rank, MPI::message::event_size, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
 
     // Reallocate if needed
-    if (mep_size > buffer_size) {
-      buffer_size = mep_size * bank_size_fudge_factor;
+    if (mep_size > slice.slice_size) {
+      slice.slice_size = mep_size * bank_size_fudge_factor;
       // Unregister memory
-      cudaCheck(cudaHostUnregister(contents));
+      Allen::host_unregister(contents);
 
       // Free memory
       MPI_Free_mem(contents);
 
       // Allocate new memory
-      MPI_Alloc_mem(buffer_size, MPI_INFO_NULL, &contents);
+      MPI_Alloc_mem(slice.slice_size, MPI_INFO_NULL, &contents);
 
       // Only bind explicitly if there are multiple receivers,
       // otherwise assume a memory allocation policy is in effect
@@ -766,7 +768,7 @@ void MEPProvider::mpi_read()
         // Bind memory to numa domain of receiving card
         auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
         auto s = hwloc_set_area_membind(
-          m_topology, contents, buffer_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
+          m_topology, contents, slice.slice_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
         if (s != 0) {
           m_read_error = true;
           error_cout << "Failed to bind memory to node " << std::to_string(numa_node) << " " << strerror(errno) << "\n";
@@ -776,13 +778,13 @@ void MEPProvider::mpi_read()
 
       // Register memory with CUDA
       try {
-        cudaCheck(cudaHostRegister(contents, buffer_size, cudaHostRegisterDefault));
+        Allen::host_register(contents, slice.slice_size, Allen::hostRegisterDefault);
       } catch (std::invalid_argument const&) {
         m_read_error = true;
         break;
       }
 
-      buffer_span = gsl::span {contents, static_cast<events_size>(buffer_size)};
+      slice.mep_data = gsl::span {contents, static_cast<events_size>(slice.slice_size)};
     }
 
     // Number of full-size (MPI::mdf_chunk_size) messages
@@ -841,7 +843,7 @@ void MEPProvider::mpi_read()
     slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(contents);
     slice.mep_data = gsl::span {contents, static_cast<events_size>(mep_size)};
 
-    size_t n_blocks = mep_header->n_MFPs;
+    auto const* mep_header = slice.mep_header;
     auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0]);
     slice.packing_factor = block_header->n_banks;
 
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index ce804c6ec..d20eaa7ee 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -26,6 +26,10 @@
 
 #include "TransposeMEP.h"
 
+#ifdef HAVE_MPI
+#include <hwloc.h>
+#endif
+
 namespace {
   using namespace Allen::Units;
   using namespace std::string_literals;
@@ -199,9 +203,6 @@ private:
 
   #ifdef HAVE_MPI
   std::vector<std::tuple<int, int>> m_domains;
-  #endif
-
-  #ifdef HAVE_HWLOC
   hwloc_topology_t m_topology;
   #endif
 
-- 
GitLab


From 9e3ae93991e01932a68ea04e52eb875d017ec9f4 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 3 Jun 2021 15:14:34 +0200
Subject: [PATCH 004/120] Follow changes in Allen master

---
 AllenOnline/src/MEPProvider.cpp  |  3 ++-
 AllenOnline/src/MEPProvider.h    |  3 +++
 AllenOnline/src/TransposeMEP.cpp | 13 +++++++++----
 AllenOnline/src/TransposeMEP.h   | 10 +++++-----
 4 files changed, 19 insertions(+), 10 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index d4a74d368..b2f6e0f1f 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -124,6 +124,7 @@ BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) cons
   }
   std::get<1>(bno) = m_config.transpose_mep ? offsets[offsets_size - 1] : data_size;
   std::get<2>(bno) = offsets;
+  std::get<3>(bno) = m_banks_version[ib];
   return bno;
 }
 
@@ -455,7 +456,7 @@ void MEPProvider::allocate_storage(size_t i_read)
     }
   }
 
-  std::tie(count_success, m_banks_count) = MEP::fill_counts(mep_header, slice.mep_data);
+  std::tie(count_success, m_banks_count, m_banks_version) = MEP::fill_counts(mep_header, slice.mep_data, m_bank_ids);
 
   // Allocate slice memory that will contain transposed banks ready
   // for processing by the Allen kernels
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index d20eaa7ee..82689d8a6 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -230,6 +230,9 @@ private:
   Slices m_slices;
   std::vector<std::tuple<int, size_t, size_t>> m_slice_to_buffer;
 
+  // Array to store the version of banks per bank type
+  mutable std::array<int, NBankTypes> m_banks_version;
+
   // Mutex, condition varaible and queue for parallel transposition of slices
   std::mutex m_transpose_mut;
   std::condition_variable m_transpose_cond;
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 9503682fb..b24e7ac74 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -17,9 +17,8 @@ namespace {
   }
 }
 
-std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>> MEP::fill_counts(
-  MEP::MEP_header const* header,
-  gsl::span<char const> const& mep_span)
+std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
+MEP::fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& mep_span, std::vector<int> const& bank_ids)
 {
   // info_cout << "EB header: "
   //   << header.n_blocks << ", "
@@ -30,6 +29,7 @@ std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>> MEP::fill_counts(
   auto header_size = header->header_size();
   gsl::span<char const> block_span {mep_span.data() + header_size, mep_span.size() - header_size};
   std::array<unsigned int, LHCb::NBankTypes> count {0};
+  std::array<int, NBankTypes> versions {0};
   for (size_t i = 0; i < header->n_MFPs; ++i) {
     auto offset = header->offsets()[i];
     auto const* bh = reinterpret_cast<MFP::MFP_header const*>(block_span.data() + offset);
@@ -42,9 +42,14 @@ std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>> MEP::fill_counts(
     if (type < LHCb::RawBank::LastType) {
       ++count[type];
     }
+
+    auto const allen_type = bank_ids[type];
+    if (allen_type != -1) {
+      versions[allen_type] = bh->block_version;
+    }
   }
 
-  return {true, count};
+  return {true, count, versions};
 }
 
 void MEP::find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor, gsl::span<char const> const& buffer_span, Blocks& blocks)
diff --git a/AllenOnline/src/TransposeMEP.h b/AllenOnline/src/TransposeMEP.h
index 079020c23..6b70f3443 100644
--- a/AllenOnline/src/TransposeMEP.h
+++ b/AllenOnline/src/TransposeMEP.h
@@ -51,14 +51,14 @@ namespace MEP {
    *
    * @details    detailed description
    *
-   * @param      MEP::MEP_header for a MEP
+   * @param      EB::Header for a MEP
    * @param      span of the block data in the MEP
+   * @param      LHCb::RawBank::BankType to Allen bank type mapping
    *
-   * @return     (success, number of banks per bank type; 0 if the bank is not needed)
+   * @return     (success, number of banks per bank type; 0 if the bank is not needed, bank version per type)
    */
-  std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>> fill_counts(
-    MEP::MEP_header const* header,
-    gsl::span<char const> const& data);
+  std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
+  fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& data, std::vector<int> const& bank_ids);
 
   void find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor, gsl::span<char const> const& buffer_span, Blocks& blocks);
 
-- 
GitLab


From ce61aec9f657c2fe0ea21cbbced98948f2e78952 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 3 Jun 2021 15:15:01 +0200
Subject: [PATCH 005/120] Build with old cmake, find Allen and set device
 runtime by hand

---
 AllenOnline/CMakeLists.txt | 16 ++++++++++++++--
 1 file changed, 14 insertions(+), 2 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 1e7b29d29..bd4b30510 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -25,12 +25,24 @@ include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
 
 include_directories(include)
 
+# More hacky way to find Allen
+find_library(ALLEN_LIBRARY
+  AllenLib
+  PATH_SUFFIXES ../lib
+  REQUIRED)
+find_package(CUDAToolkit REQUIRED)
+
+set(TARGET_DEVICE "CUDA")
+set(TARGET_DEFINITION "TARGET_DEVICE_${TARGET_DEVICE}")
+add_compile_definitions(${TARGET_DEFINITION})
+
 gaudi_add_library(AllenOnline
                  src/ReadMEP.cpp
                  src/TransposeMEP.cpp
                  NO_PUBLIC_HEADERS
                  INCLUDE_DIRS GaudiKernel cppgsl Online/OnlineKernel Online/EventBuilding
-                 LINK_LIBRARIES GaudiKernel Allen::AllenLib EventBuildingLib)
+                 LINK_LIBRARIES GaudiKernel ${ALLEN_LIBRARY} EventBuildingLib CUDA::cudart)
+
 
 gaudi_add_module(AllenOnlineComp
                  src/AllenApplication.cpp
@@ -38,7 +50,7 @@ gaudi_add_module(AllenOnlineComp
                  src/MEPProvider.cpp
                  INCLUDE_DIRS OnlineBase ROOT Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
                  LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQLib RPC Parsers EventBuildingLib MDFLib
-                                OnlineKernel Allen::AllenLib Allen::BinaryDumpers Allen::AllenRuntime DAQEventLib
+                                OnlineKernel ${ALLEN_LIBRARY} BinaryDumpers DAQEventLib
                                 DAQKernelLib GaudiAlgLib PrKernel VPDetLib UTDetLib UTKernelLib AllenOnline
                                 FTDetLib -lrt)
 
-- 
GitLab


From 4b88a40714007aa73828822798710ef85ff305c6 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 4 Jun 2021 11:29:26 +0200
Subject: [PATCH 006/120] Fix Allen.py for AllenApplication

---
 AllenOnline/options/Allen.py | 19 ++++++++-----------
 1 file changed, 8 insertions(+), 11 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 2f9769595..4afc546d0 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -6,6 +6,7 @@ from Configurables import DumpMagneticField, DumpBeamline, DumpUTLookupTables
 from Configurables import AllenConfiguration
 from Configurables import MonitorSvc
 from Configurables import Online__Configuration as OnlineConfiguration
+from Allen.config import setup_allen_non_event_data_service
 
 app = LHCbApp(
     DataType="Upgrade",
@@ -17,17 +18,11 @@ app = LHCbApp(
 # Upgrade DBs
 CondDB().Upgrade = True
 
-producers = [
-    p(DumpToFile=False)
-    for p in (DumpVPGeometry, DumpUTGeometry, DumpFTGeometry, DumpMuonGeometry,
-              DumpMuonTable, DumpMagneticField, DumpBeamline,
-              DumpUTLookupTables)
-]
 
 online_conf = OnlineConfiguration("Application")
 online_conf.debug = False
 online_conf.classType = 1
-online_conf.automatic = False
+online_conf.autoStart = False
 online_conf.monitorType = 'MonitorSvc'
 online_conf.logDeviceType = 'RTL::Logger::LogDevice'
 online_conf.logDeviceFormat = '%TIME%LEVEL%-8NODE: %-32PROCESS %-20SOURCE'
@@ -38,7 +33,8 @@ allen_conf = AllenConfiguration()
 allen_conf.EventsPerSlice = 1000
 allen_conf.NonStop = True
 allen_conf.MPI = False
-allen_conf.Receivers = "mlx5_0:1"
+allen_conf.JSON = "/daqarea1/fest/allen/configuration/config.json"
+allen_conf.Receivers = {"mlx5_0": 1}
 allen_conf.NThreads = 8
 allen_conf.NSlices = 16
 # allen_conf.Output = "tcp://192.168.1.101:35000"
@@ -55,11 +51,12 @@ monSvc.ExpandNameInfix = '<proc>'
 monSvc.ExpandCounterServices = True
 monSvc.UniqueServiceNames = True
 
+setup_allen_non_event_data_service()
+
 # Add the services that will produce the non-event-data
 ApplicationMgr().ExtSvc += [
-    monSvc,
-    AllenUpdater(OutputLevel=2),
-] + producers
+    monSvc
+]
 
 # Some extra stuff for timing table
 ApplicationMgr().EvtSel = "NONE"
-- 
GitLab


From 612aed82b56a2444b8a616921dcf6c49cb7c74fd Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 4 Jun 2021 11:30:14 +0200
Subject: [PATCH 007/120] WIP updating mep reading

---
 AllenOnline/CMakeLists.txt           |  5 ++
 AllenOnline/application/read_mep.cpp | 83 ++++++++++++++++++++++++++++
 AllenOnline/src/ReadMEP.cpp          | 54 +++++++-----------
 3 files changed, 108 insertions(+), 34 deletions(-)
 create mode 100644 AllenOnline/application/read_mep.cpp

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index bd4b30510..03d645ad8 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -80,4 +80,9 @@ if (MPI_FOUND)
                        INCLUDE_DIRS GaudiKernel ${MPI_CXX_INCLUDE_DIRS}
                        LINK_LIBRARIES AllenOnline MDFLib Boost::program_options MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
   target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
+
+  gaudi_add_executable(test_read_mep
+                       application/read_mep.cpp
+                       INCLUDE_DIRS GaudiKernel
+                       LINK_LIBRARIES AllenOnline MDFLib Boost::program_options)
 endif()
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
new file mode 100644
index 000000000..4516a2ff1
--- /dev/null
+++ b/AllenOnline/application/read_mep.cpp
@@ -0,0 +1,83 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <cstring>
+#include <iostream>
+#include <fstream>
+#include <string>
+#include <iomanip>
+#include <unordered_set>
+#include <map>
+#include <cassert>
+
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <unistd.h>
+
+#include <Allen/Logger.h>
+
+#include <Event/RawBank.h>
+
+#include <MDF/StreamDescriptor.h>
+#include <ReadMEP.hpp>
+
+using namespace std;
+
+int main(int argc, char* argv[])
+{
+  if (argc != 3) {
+    cout << "usage: read_mep file.mep n_mep" << endl;
+    return -1;
+  }
+
+  string filename = {argv[1]};
+  size_t n_meps = atol(argv[2]);
+
+  // Some storage for reading the events into
+  bool eof = false, success = false;
+  EB::Header mep_header;
+  gsl::span<char const> mep_span;
+
+  LHCb::StreamDescriptor::connect(*m_current);
+  auto input = MDF::open(filename, O_RDONLY);
+  if (input.good) {
+    info_cout << "Opened " << filename << "\n";
+  }
+  else {
+    cerr << "Failed to open file " << filename << " " << strerror(errno) << "\n";
+    return -1;
+  }
+
+  vector<char> data;
+
+  size_t i_mep = 0;
+  while (!eof && i_mep++ < n_meps) {
+
+    std::tie(eof, success, mep_header, mep_span) = MEP::read_mep(input, data);
+
+    auto header_size = mep_header.header_size(mep_header.n_blocks);
+    auto const* d = mep_span.begin() + header_size;
+    size_t i_block = 0;
+    while (d != mep_span.end()) {
+      EB::BlockHeader const block_header {d};
+      char const* block_data = d + block_header.header_size(block_header.n_frag);
+      char const* block_end = block_data + block_header.block_size;
+
+      assert(d - (mep_span.begin() + header_size) == mep_header.offsets[i_block]);
+
+      auto lhcb_type = int {block_header.types[0]};
+
+      cout << "fragment"
+           << " packing: " << std::setw(4) << block_header.n_frag << " event_id: " << std::setw(6)
+           << block_header.event_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
+           << mep_header.source_ids[i_block] << " version: " << std::setw(2) << mep_header.versions[i_block]
+           << " size: " << std::setw(6) << block_header.block_size << "\n";
+
+      d = block_end;
+      ++i_block;
+    }
+  }
+
+  return 0;
+}
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index acbaecbf7..672316eb6 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -37,11 +37,11 @@ namespace {
 std::tuple<bool, bool, MEP::MEP_header const*, unsigned, gsl::span<char const>>
 MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
 {
+  // Allocate space for the first few words of the MEP header
+  buffer.resize(sizeof(MEP::MEP_header));
 
-  buffer.resize(sizeof(LHCb::MDFHeader));
-  LHCb::MDFHeader* mdf_header = reinterpret_cast<LHCb::MDFHeader*>(buffer.data());
-
-  ssize_t n_bytes = input.ioFuncs->read(input.ioDesc, &buffer[0], sizeof(LHCb::MDFHeader));
+  // Read the first few words of the MEP header
+  ssize_t n_bytes = input.ioFuncs->read(input.ioDesc, &buffer[0], sizeof(MEP::MEP_header));
   if (n_bytes == 0) {
     cout << "Cannot read more data (Header). End-of-File reached.\n";
     return {true, true, nullptr, 0u, {}};
@@ -50,48 +50,34 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
     cerr << "Failed to read header " << strerror(errno) << "\n";
     return {false, false, nullptr, 0u, {}};
   }
-  unsigned header_version = mdf_header->headerVersion();
-  auto hdr_size = LHCb::MDFHeader::sizeOf(header_version);
-  assert((hdr_size - sizeof(LHCb::MDFHeader)) == mdf_header->subheaderLength());
-  // read subheader
-  buffer.resize(hdr_size + sizeof(MEP::MEP_header));
-  mdf_header = reinterpret_cast<LHCb::MDFHeader*>(&buffer[0]);
-  n_bytes = input.read(&buffer[0] + sizeof(LHCb::MDFHeader), mdf_header->subheaderLength());
-  if (n_bytes <= 0) {
-    cerr << "Failed to read subheader " << strerror(errno) << "\n";
-    return {false, false, nullptr, 0u, {}};
-  }
 
-  // read MEP_header
-  char* mep_buffer = &buffer[0] + hdr_size;
-  MEP::MEP_header* mep_header = reinterpret_cast<MEP::MEP_header*>(mep_buffer);
-  n_bytes = input.read(mep_buffer, sizeof(MEP::MEP_header));
-  if (n_bytes <= 0) {
-    cerr << "Failed to MEP_header base " << strerror(errno) << "\n";
-    return {false, false, nullptr, 0u, {}};
+  // Check magic pattern
+  MEP::MEP_header const* mep_header = reinterpret_cast<MEP::MEP_header const*>(buffer.data());
+  if (mep_header->magic != 0xFACE) {
+    cerr << "Wrong magic pattern in MEP header: " << mep_header->magic << "\n";
+    return {true, true, nullptr, 0u, {}};
   }
 
-  buffer.resize(hdr_size + mep_header->header_size());
-  mep_buffer = &buffer[0] + hdr_size;
-  mep_header = reinterpret_cast<MEP::MEP_header*>(mep_buffer);
+  // The size of the MEP has been read, so allocate space for that
+  // (don't forget to redo the pointers in case the memory was
+  // reallocated elsewhere)
   auto data_size = static_cast<size_t>(mep_header->bytes());
+  buffer.resize(data_size);
+  char* mep_buffer = &buffer[0];
+  mep_header = reinterpret_cast<MEP::MEP_header const*>(mep_buffer);
 
-  buffer.resize(hdr_size + mep_header->header_size() + data_size);
-  mdf_header = reinterpret_cast<LHCb::MDFHeader*>(&buffer[0]);
-  mep_buffer = &buffer[0] + hdr_size;
-  mep_header = reinterpret_cast<MEP::MEP_header*>(mep_buffer);
-
+  // Read the rest of the MEP now that the size is known
   n_bytes = input.ioFuncs->read(input.ioDesc,
-    mep_buffer + mep_header->header_size(),
-    static_cast<int>(mep_header->header_size() - sizeof(MEP::MEP_header) + data_size));
+    mep_buffer + sizeof(MEP::MEP_header),
+    static_cast<int>(data_size - sizeof(MEP::MEP_header)));
   if (n_bytes <= 0) {
     cerr << "Failed to read MEP" << strerror(errno) << "\n";
     return {false, false, nullptr, 0u, {}};
   }
 
+  // Get the packing factor
   auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0]);
 
-  auto total_size = mep_header->header_size() + data_size;
   return {false, true, mep_header, block_header->n_banks,
-          {buffer.data() + hdr_size, total_size}};
+          {buffer.data(), data_size}};
 }
-- 
GitLab


From a20b06b97887cafefcc13574f1368fc7b0237e06 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 4 Jun 2021 14:30:11 +0200
Subject: [PATCH 008/120] Follow changes in Allen and move MEPProvider to
 public header

---
 AllenOnline/{src => AllenOnline}/MEPProvider.h  |  2 +-
 AllenOnline/{src => AllenOnline}/TransposeMEP.h |  8 ++++----
 AllenOnline/CMakeLists.txt                      |  2 +-
 AllenOnline/src/AllenApplication.cpp            |  3 ++-
 AllenOnline/src/MEPProvider.cpp                 |  7 ++++---
 AllenOnline/src/TransposeMEP.cpp                | 11 ++++++-----
 6 files changed, 18 insertions(+), 15 deletions(-)
 rename AllenOnline/{src => AllenOnline}/MEPProvider.h (99%)
 rename AllenOnline/{src => AllenOnline}/TransposeMEP.h (97%)

diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
similarity index 99%
rename from AllenOnline/src/MEPProvider.h
rename to AllenOnline/AllenOnline/MEPProvider.h
index 82689d8a6..94d0898e4 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -227,7 +227,7 @@ private:
   std::vector<int> m_bank_ids;
 
   // Memory slices, N for each raw bank type
-  Slices m_slices;
+  Allen::Slices m_slices;
   std::vector<std::tuple<int, size_t, size_t>> m_slice_to_buffer;
 
   // Array to store the version of banks per bank type
diff --git a/AllenOnline/src/TransposeMEP.h b/AllenOnline/AllenOnline/TransposeMEP.h
similarity index 97%
rename from AllenOnline/src/TransposeMEP.h
rename to AllenOnline/AllenOnline/TransposeMEP.h
index 6b70f3443..40e43fe7c 100644
--- a/AllenOnline/src/TransposeMEP.h
+++ b/AllenOnline/AllenOnline/TransposeMEP.h
@@ -65,7 +65,7 @@ namespace MEP {
   void fragment_offsets(Blocks const& blocks, std::vector<std::vector<uint32_t>>& offsets);
 
   size_t allen_offsets(
-    ::Slices& slices,
+    Allen::Slices& slices,
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
@@ -76,7 +76,7 @@ namespace MEP {
     bool split_by_run = false);
 
   std::tuple<bool, bool, size_t> mep_offsets(
-    ::Slices& slices,
+    Allen::Slices& slices,
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
@@ -99,7 +99,7 @@ namespace MEP {
    * @return     tuple of: (success, slice is full)
    */
   bool transpose_event(
-    ::Slices& slices,
+    Allen::Slices& slices,
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
@@ -122,7 +122,7 @@ namespace MEP {
    * @return     tuple of: (success, slice is full)
    */
   std::tuple<bool, bool, size_t> transpose_events(
-    ::Slices& slices,
+    Allen::Slices& slices,
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 03d645ad8..2aa760c4f 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -39,7 +39,7 @@ add_compile_definitions(${TARGET_DEFINITION})
 gaudi_add_library(AllenOnline
                  src/ReadMEP.cpp
                  src/TransposeMEP.cpp
-                 NO_PUBLIC_HEADERS
+                 PUBLIC_HEADERS AllenOnline
                  INCLUDE_DIRS GaudiKernel cppgsl Online/OnlineKernel Online/EventBuilding
                  LINK_LIBRARIES GaudiKernel ${ALLEN_LIBRARY} EventBuildingLib CUDA::cudart)
 
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 2090f0647..e3d996a47 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -27,13 +27,14 @@
 #include <Allen/BankMapping.h>
 #include <Allen/Allen.h>
 
+#include <AllenOnline/MEPProvider.h>
+
 #ifdef HAVE_MPI
 #include <MPIConfig.h>
 #endif
 
 #include "AllenConfiguration.h"
 #include "AllenApplication.h"
-#include "MEPProvider.h"
 
 // #include "EBProvider.h"
 
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index b2f6e0f1f..e2e1198b7 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -14,7 +14,9 @@
 
 #include <Event/RawBank.h>
 
-#include "TransposeMEP.h"
+#include "AllenOnline/MEPProvider.h"
+#include "AllenOnline/TransposeMEP.h"
+
 #include "WriteMDF.h"
 #include "ReadMEP.h"
 
@@ -23,7 +25,6 @@
 #include <hwloc.h>
 #endif
 
-#include "MEPProvider.h"
 
 
 MEPProvider::MEPProvider(
@@ -67,7 +68,7 @@ InputProvider {n_slices, events_per_slice, bank_types,
   }
 
   // Cached bank LHCb bank type to Allen bank type mapping
-  m_bank_ids = bank_ids();
+  m_bank_ids = Allen::bank_ids();
 
   // Reserve 1MB for decompression
   m_compress_buffer.reserve(1u * MB);
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index b24e7ac74..5b8b1275f 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -3,10 +3,11 @@
 \*****************************************************************************/
 #include <cassert>
 #include <cstring>
-#include "TransposeMEP.h"
 
 #include <Event/ODIN.h>
 
+#include <AllenOnline/TransposeMEP.h>
+
 namespace {
   std::tuple<unsigned, unsigned long> decode_odin(unsigned const* odin_data) {
     unsigned const run_number = odin_data[LHCb::ODIN::Data::RunNumber];
@@ -90,7 +91,7 @@ void MEP::fragment_offsets(MEP::Blocks const& blocks, MEP::SourceOffsets& offset
 }
 
 size_t MEP::allen_offsets(
-  ::Slices& slices,
+  Allen::Slices& slices,
   int const slice_index,
   std::vector<int> const& bank_ids,
   std::unordered_set<BankTypes> const& bank_types,
@@ -172,7 +173,7 @@ size_t MEP::allen_offsets(
 }
 
 std::tuple<bool, bool, size_t> MEP::mep_offsets(
-  ::Slices& slices,
+  Allen::Slices& slices,
   int const slice_index,
   std::vector<int> const& bank_ids,
   std::unordered_set<BankTypes> const& bank_types,
@@ -268,7 +269,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets(
 }
 
 bool MEP::transpose_event(
-  ::Slices& slices,
+  Allen::Slices& slices,
   int const slice_index,
   std::vector<int> const& bank_ids,
   std::unordered_set<BankTypes> const& bank_types,
@@ -358,7 +359,7 @@ bool MEP::transpose_event(
 }
 
 std::tuple<bool, bool, size_t> MEP::transpose_events(
-  ::Slices& slices,
+  Allen::Slices& slices,
   int const slice_index,
   std::vector<int> const& bank_ids,
   std::unordered_set<BankTypes> const& bank_types,
-- 
GitLab


From 0fa786fc0f84c553aa7a09f67165fbe0f7575162 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 4 Jun 2021 17:49:47 +0200
Subject: [PATCH 009/120] Fixes to MEP reading.

---
 AllenOnline/application/MPISend.cpp  |  2 +-
 AllenOnline/application/read_mep.cpp | 52 +++++++++++++++-------------
 AllenOnline/src/MEPProvider.cpp      |  4 +--
 AllenOnline/src/ReadMEP.cpp          | 13 ++++---
 AllenOnline/src/TransposeMEP.cpp     | 18 ++++------
 5 files changed, 43 insertions(+), 46 deletions(-)

diff --git a/AllenOnline/application/MPISend.cpp b/AllenOnline/application/MPISend.cpp
index e22ef15f2..9cbbcf1e8 100644
--- a/AllenOnline/application/MPISend.cpp
+++ b/AllenOnline/application/MPISend.cpp
@@ -96,7 +96,7 @@ int main(int argc, char* argv[]) {
     MEP::MEP_header const* mep_header = nullptr;
     unsigned pf = 0;
 
-    auto input = LHCb::StreamDescriptor::connect(connection);
+    auto input = LHCb::StreamDescriptor::bind(connection);
     if (input.ioDesc != 0) {
       std::cout << "Opened " << connection << "\n";
     }
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 4516a2ff1..a648a8083 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -15,12 +15,16 @@
 #include <fcntl.h>
 #include <unistd.h>
 
-#include <Allen/Logger.h>
-
 #include <Event/RawBank.h>
 
+#include <Allen/Logger.h>
+
 #include <MDF/StreamDescriptor.h>
-#include <ReadMEP.hpp>
+
+#include <EventBuilding/MEP_tools.hpp>
+#include <EventBuilding/MFP_tools.hpp>
+
+#include <ReadMEP.h>
 
 using namespace std;
 
@@ -36,12 +40,11 @@ int main(int argc, char* argv[])
 
   // Some storage for reading the events into
   bool eof = false, success = false;
-  EB::Header mep_header;
+  MEP::MEP_header const* mep_header;
   gsl::span<char const> mep_span;
 
-  LHCb::StreamDescriptor::connect(*m_current);
-  auto input = MDF::open(filename, O_RDONLY);
-  if (input.good) {
+  auto input = LHCb::StreamDescriptor::bind(filename);
+  if (input.ioDesc != 0) {
     info_cout << "Opened " << filename << "\n";
   }
   else {
@@ -50,32 +53,31 @@ int main(int argc, char* argv[])
   }
 
   vector<char> data;
+  unsigned packing_factor = 0;
 
   size_t i_mep = 0;
   while (!eof && i_mep++ < n_meps) {
 
-    std::tie(eof, success, mep_header, mep_span) = MEP::read_mep(input, data);
-
-    auto header_size = mep_header.header_size(mep_header.n_blocks);
-    auto const* d = mep_span.begin() + header_size;
-    size_t i_block = 0;
-    while (d != mep_span.end()) {
-      EB::BlockHeader const block_header {d};
-      char const* block_data = d + block_header.header_size(block_header.n_frag);
-      char const* block_end = block_data + block_header.block_size;
+    std::tie(eof, success, mep_header, packing_factor, mep_span) = MEP::read_mep(input, data);
+    if (!success) {
+      return 1;
+    }
+    for (uint16_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
+      // block offsets are in number of 4-byte words
+      auto const block_offset = mep_header->offsets()[i_block] * sizeof(uint32_t);
+      auto block_header = reinterpret_cast<MFP::MFP_header const*>(mep_span.data() + block_offset);
+      char const* block_data = static_cast<char const*>(block_header->payload());
+      [[maybe_unused]] char const* block_end = block_data + block_header->bytes() - block_header->header_size();
 
-      assert(d - (mep_span.begin() + header_size) == mep_header.offsets[i_block]);
+      assert(static_cast<size_t>(block_end - mep_span.data()) <= mep_header->offsets()[i_block + 1] * sizeof(uint32_t));
 
-      auto lhcb_type = int {block_header.types[0]};
+      auto lhcb_type = int {block_header->bank_types()[0]};
 
       cout << "fragment"
-           << " packing: " << std::setw(4) << block_header.n_frag << " event_id: " << std::setw(6)
-           << block_header.event_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
-           << mep_header.source_ids[i_block] << " version: " << std::setw(2) << mep_header.versions[i_block]
-           << " size: " << std::setw(6) << block_header.block_size << "\n";
-
-      d = block_end;
-      ++i_block;
+           << " packing: " << std::setw(4) << block_header->n_banks << " event_id: " << std::setw(6)
+           << block_header->ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
+           << (block_header->src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{block_header->block_version}
+           << " size: " << std::setw(6) << block_header->bytes() << "\n";
     }
   }
 
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index e2e1198b7..7b6fde813 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -269,7 +269,7 @@ void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event,
     offset += add_raw_bank(
       block_header->bank_types()[mep_event],
       block_header->block_version,
-      slice.mep_header->src_ids()[i_block],
+      block_header->src_id & 0x7FF,
       {block_data.data() + fragment_offset, fragment_size},
       buffer.data() + offset);
     ++block_index;
@@ -511,7 +511,7 @@ bool MEPProvider::open_file() const
 
     if (m_input) m_input->close();
 
-    m_input = LHCb::StreamDescriptor::connect(*m_current);
+    m_input = LHCb::StreamDescriptor::bind(*m_current);
     if (m_input->ioDesc != 0) {
       info_cout << "Opened " << *m_current << "\n";
       good = true;
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index 672316eb6..a2a06ba82 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -41,7 +41,7 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   buffer.resize(sizeof(MEP::MEP_header));
 
   // Read the first few words of the MEP header
-  ssize_t n_bytes = input.ioFuncs->read(input.ioDesc, &buffer[0], sizeof(MEP::MEP_header));
+  auto n_bytes = input.read(&buffer[0], sizeof(MEP::MEP_header));
   if (n_bytes == 0) {
     cout << "Cannot read more data (Header). End-of-File reached.\n";
     return {true, true, nullptr, 0u, {}};
@@ -53,9 +53,9 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
 
   // Check magic pattern
   MEP::MEP_header const* mep_header = reinterpret_cast<MEP::MEP_header const*>(buffer.data());
-  if (mep_header->magic != 0xFACE) {
-    cerr << "Wrong magic pattern in MEP header: " << mep_header->magic << "\n";
-    return {true, true, nullptr, 0u, {}};
+  if (mep_header->magic != 0xCEFA) {
+    cerr << "Wrong magic pattern in MEP header: 0x" << std::hex << mep_header->magic << "\n";
+    return {false, false, nullptr, 0u, {}};
   }
 
   // The size of the MEP has been read, so allocate space for that
@@ -67,8 +67,7 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   mep_header = reinterpret_cast<MEP::MEP_header const*>(mep_buffer);
 
   // Read the rest of the MEP now that the size is known
-  n_bytes = input.ioFuncs->read(input.ioDesc,
-    mep_buffer + sizeof(MEP::MEP_header),
+  n_bytes = input.read(mep_buffer + sizeof(MEP::MEP_header),
     static_cast<int>(data_size - sizeof(MEP::MEP_header)));
   if (n_bytes <= 0) {
     cerr << "Failed to read MEP" << strerror(errno) << "\n";
@@ -76,7 +75,7 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   }
 
   // Get the packing factor
-  auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0]);
+  auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0] * sizeof(uint32_t));
 
   return {false, true, mep_header, block_header->n_banks,
           {buffer.data(), data_size}};
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 5b8b1275f..05441d170 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -37,8 +37,6 @@ MEP::fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& mep
 
     // info_cout << "EB BlockHeader: "
     //   << bh.event_id << ", " << bh.n_frag << ", " << bh.reserved << ", " << bh.block_size << "\n";
-
-    assert(bh->n_frag != 0);
     auto type = bh->bank_types()[0];
     if (type < LHCb::RawBank::LastType) {
       ++count[type];
@@ -57,14 +55,13 @@ void MEP::find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor
 {
 
   // Fill blocks
-  auto hdr_size = mep_header->header_size();
   auto block_hdr_size = MFP::MFP_header_size(packing_factor, MFP::MFP_aligment);
-  gsl::span<char const> const mep_data {buffer_span.data() + hdr_size, buffer_span.size() - hdr_size};
 
   for (size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
-    auto block_offset = mep_header->offsets()[i_block];
-    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>(mep_data.data() + block_offset);
-    gsl::span<char const> block_data {mep_data.data() + block_offset + block_hdr_size, block_header->bytes()};
+    // block offsets are in 4-byte words with respect to the start of the MEP header
+    auto block_offset = mep_header->offsets()[i_block] * sizeof(uint32_t);
+    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>(buffer_span.data() + block_offset);
+    gsl::span<char const> block_data {buffer_span.data() + block_offset + block_hdr_size, block_header->bytes()};
     blocks[i_block] = std::tuple {block_header, std::move(block_data)};
   }
 }
@@ -179,7 +176,6 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets(
   std::unordered_set<BankTypes> const& bank_types,
   std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
   EventIDs& event_ids,
-  MEP::MEP_header const* mep_header,
   MEP::Blocks const& blocks,
   std::tuple<size_t, size_t> const& interval,
   bool split_by_run)
@@ -240,8 +236,8 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets(
         prev_type = lhcb_type;
       }
 
-      // Store source ID
-      event_offsets[2 + offset_index] = mep_header->src_ids()[i_block];
+      // Store source ID, mask top 5 bits which are detector ID
+      event_offsets[2 + offset_index] = block_header->src_id & 0x7FF;
 
       // Initialize the first offsets using the block sizes,
       if (offset_index < banks_count[lhcb_type] - 1) {
@@ -346,7 +342,7 @@ bool MEP::transpose_event(
 
         // Write sourceID; offset in 32bit words
         auto word_offset = offset / sizeof(uint32_t);
-        banks_write[word_offset] = mep_header->src_ids()[i_block];
+        banks_write[word_offset] = block_header->src_id & 0x7FF;
 
         // Write bank data
         std::memcpy(banks_write + word_offset + 1, block_data.data() + source_offsets[i_event], frag_size);
-- 
GitLab


From 16b45da8586aa0a76c34312ac3a08bbead752e0e Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Sun, 6 Jun 2021 09:57:27 +0200
Subject: [PATCH 010/120] Remove EBProvider because it wasn't used.

---
 AllenOnline/src/EBProvider.h | 1293 ----------------------------------
 1 file changed, 1293 deletions(-)
 delete mode 100644 AllenOnline/src/EBProvider.h

diff --git a/AllenOnline/src/EBProvider.h b/AllenOnline/src/EBProvider.h
deleted file mode 100644
index 43ef89f07..000000000
--- a/AllenOnline/src/EBProvider.h
+++ /dev/null
@@ -1,1293 +0,0 @@
-/*****************************************************************************\
-* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
-\*****************************************************************************/
-#pragma once
-
-#include <thread>
-#include <vector>
-#include <array>
-#include <deque>
-#include <mutex>
-#include <atomic>
-#include <chrono>
-#include <algorithm>
-#include <numeric>
-#include <condition_variable>
-#include <cassert>
-
-#include <unistd.h>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <fcntl.h>
-
-#include <InputProvider.h>
-#include <BankTypes.h>
-#include <Event/RawBank.h>
-#include <MEPTools.h>
-#include <SliceUtils.h>
-
-#ifdef HAVE_HWLOC
-#include <hwloc.h>
-#endif
-
-namespace {
-  using namespace Allen::Units;
-  using namespace std::string_literals;
-} // namespace
-
-/**
- * @brief      Configuration parameters for the MEPProvider
- */
-struct EBProviderConfig {
-  // check the MDF checksum if it is available
-  bool check_checksum = false;
-
-  // number of prefetch buffers
-  size_t n_buffers = 8;
-
-  // number of transpose threads
-  size_t n_transpose_threads = 5;
-
-  int window_size = 4;
-
-  // Use MPI and number of receivers
-  bool mpi = false;
-
-  bool non_stop = true;
-
-  bool transpose_mep = false;
-
-  bool split_by_run = false;
-
-  size_t n_receivers() const { return receivers.size(); }
-
-  // Mapping of receiver card to MPI rank to receive from
-  std::map<std::string, int> receivers;
-};
-
-/**
- * @brief      Provide transposed events from MDF files
- *
- * @details    The provider has three main components
- *             - a prefetch thread to read from the current input
- *               file into prefetch buffers
- *             - N transpose threads that read from prefetch buffers
- *               and fill the per-bank-type slices with transposed sets
- *               of banks and the offsets to individual bank inside a
- *               given set
- *             - functions to obtain a transposed slice and declare it
- *               for refilling
- *
- *             Access to prefetch buffers and slices is synchronised
- *             using mutexes and condition variables.
- *
- * @param      Number of slices to fill
- * @param      Number of events per slice
- * @param      MDF filenames
- * @param      Configuration struct
- *
- */
-template<BankTypes... Banks>
-class EBProvider final : public InputProvider<MEPProvider<Banks...>> {
-public:
-  MEPProvider(
-    size_t n_slices,
-    size_t events_per_slice,
-    std::optional<size_t> n_events,
-    std::vector<std::string> connections,
-    std::unordered_set<BankTypes> const& bank_types,
-    MEPProviderConfig config = MEPProviderConfig {}) noexcept(false) :
-    InputProvider n_slices, events_per_slice, bank_types,
-      config.transpose_mep ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP,
-      n_events},
-    m_buffer_status(config.n_buffers), m_slice_free(n_slices, true), m_banks_count {0}, m_event_ids {n_slices},
-    m_connections {std::move(connections)}, m_config {config}
-  {
-
-    if (m_config.transpose_mep) {
-      info_cout << "Providing events in Allen layout by transposing MEPs\n";
-    }
-    else {
-      info_cout << "Providing events in MEP layout\n";
-    }
-
-    m_buffer_transpose = m_buffer_status.begin();
-    m_buffer_reading = m_buffer_status.begin();
-
-    if (m_config.mpi) {
-      init_mpi();
-    }
-    else {
-      m_read_buffers.resize(m_config.n_buffers);
-      m_net_slices.resize(m_config.n_buffers);
-    }
-
-    // Initialize the current input filename
-    m_current = m_connections.begin();
-
-    // Allocate space to store event ids
-    for (size_t n = 0; n < n_slices; ++n) {
-      m_event_ids[n].reserve(events_per_slice);
-    }
-
-    // Cached bank LHCb bank type to Allen bank type mapping
-    m_bank_ids = bank_ids();
-
-    // Reserve 1MB for decompression
-    m_compress_buffer.reserve(1u * MB);
-
-    // start MPI receive or MEP reading thread
-    if (m_config.mpi) {
-#ifdef HAVE_MPI
-      m_input_thread = std::thread {&MEPProvider::mpi_read, this};
-#endif
-    }
-    else {
-      m_input_thread = std::thread {&MEPProvider::mep_read, this};
-    }
-
-    // Sanity check on the number of buffers and threads
-    if (m_config.n_buffers <= 1) {
-      warning_cout << "too few read buffers requested, setting it to 2\n";
-      m_config.n_buffers = 2;
-    }
-
-    if (m_config.n_transpose_threads > m_config.n_buffers - 1) {
-      warning_cout << "too many transpose threads requested with respect "
-                      "to the number of read buffers; reducing the number of threads to "
-                   << m_config.n_buffers - 1;
-      m_config.n_transpose_threads = m_config.n_buffers - 1;
-    }
-
-    // Start the transpose threads
-    if (m_transpose_threads.empty() && !m_read_error) {
-      for (size_t i = 0; i < m_config.n_transpose_threads; ++i) {
-        m_transpose_threads.emplace_back([this, i] { transpose(i); });
-      }
-    }
-  }
-
-  /// Destructor
-  virtual ~MEPProvider() noexcept(false)
-  {
-
-    // Set flag to indicate the prefetch thread should exit, wake it
-    // up and join it
-    m_done = true;
-    m_transpose_done = true;
-    m_mpi_cond.notify_one();
-    m_control_cond.notify_all();
-    m_input_thread.join();
-
-    // Set a flat to indicate all transpose threads should exit, wake
-    // them up and join the threads. Ensure any waiting calls to
-    // get_slice also return.
-    m_mpi_cond.notify_all();
-    m_transpose_cond.notify_all();
-    m_slice_cond.notify_all();
-
-    for (auto& thread : m_transpose_threads) {
-      thread.join();
-    }
-
-#ifdef HAVE_MPI
-    for (auto* buf : m_mpi_buffers) {
-      cudaCheck(cudaHostUnregister(buf));
-      MPI_Free_mem(buf);
-    }
-#ifdef HAVE_HWLOC
-    if (m_config.mpi) {
-      hwloc_topology_destroy(m_topology);
-    }
-#endif
-#endif
-  }
-
-  /**
-   * @brief      Obtain event IDs of events stored in a given slice
-   *
-   * @param      slice index
-   *
-   * @return     EventIDs of events in given slice
-   */
-  EventIDs event_ids(size_t slice_index, std::optional<size_t> first = {}, std::optional<size_t> last = {})
-    const override
-  {
-    auto const& ids = m_event_ids[slice_index];
-    return {ids.begin() + (first ? *first : 0), ids.begin() + (last ? *last : ids.size())};
-  }
-
-  /**
-   * @brief      Obtain banks from a slice
-   *
-   * @param      BankType
-   * @param      slice index
-   *
-   * @return     Banks and their offsets
-   */
-  BanksAndOffsets banks(BankTypes bank_type, size_t slice_index) const override
-  {
-    auto ib = to_integral<BankTypes>(bank_type);
-    auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
-
-    BanksAndOffsets bno;
-    auto& spans = std::get<0>(bno);
-    spans.reserve(banks.size());
-    for (auto s : banks) {
-      spans.emplace_back(s);
-    }
-    std::get<1>(bno) = m_config.transpose_mep ? offsets[offsets_size - 1] : data_size;
-    std::get<2>(bno) = offsets;
-    return bno;
-  }
-
-  /**
-   * @brief      Get a slice that is ready for processing; thread-safe
-   *
-   * @param      optional timeout
-   *
-   * @return     (good slice, timed out, slice index, number of events in slice)
-   */
-  std::tuple<bool, bool, bool, size_t, size_t, uint> get_slice(
-    std::optional<unsigned int> timeout = {}) override
-  {
-    bool timed_out = false, done = false;
-    size_t slice_index = 0, n_filled = 0;
-    uint run_no = 0;
-    std::unique_lock<std::mutex> lock {m_transpose_mut};
-
-    if (!m_read_error) {
-      // If no transposed slices are ready for processing, wait until
-      // one is; use a timeout if requested
-      if (m_transposed.empty()) {
-        auto wakeup = [this] {
-          auto n_writable = count_writable();
-          return (
-            !m_transposed.empty() || m_read_error || (m_transpose_done && n_writable == m_buffer_status.size()) ||
-            (m_stopping && n_writable == m_buffer_status.size()));
-        };
-        if (timeout) {
-          timed_out = !m_transpose_cond.wait_for(lock, std::chrono::milliseconds {*timeout}, wakeup);
-        }
-        else {
-          m_transpose_cond.wait(lock, wakeup);
-        }
-      }
-      if (!m_read_error && !m_transposed.empty() && (!timeout || (timeout && !timed_out))) {
-        std::tie(slice_index, n_filled) = m_transposed.front();
-        m_transposed.pop_front();
-        if (n_filled > 0) {
-          run_no = std::get<0>(m_event_ids[slice_index].front());
-        }
-      }
-    }
-
-    // Check if I/O and transposition is done and return a slice index
-    auto n_writable = count_writable();
-    done = ((m_transpose_done && m_transposed.empty()) || m_stopping) && n_writable == m_buffer_status.size();
-
-    if (timed_out && logger::verbosity() >= logger::verbose) {
-      this->debug_output(
-        "get_slice timed out; error " + std::to_string(m_read_error) + " done " + std::to_string(done) + " n_filled " +
-        std::to_string(n_filled));
-    }
-    else if (!timed_out) {
-      this->debug_output(
-        "get_slice returning " + std::to_string(slice_index) + "; error " + std::to_string(m_read_error) + " done " +
-        std::to_string(done) + " n_filled " + std::to_string(n_filled));
-    }
-
-    return {!m_read_error, done, timed_out, slice_index, m_read_error ? 0 : n_filled, run_no};
-  }
-
-  /**
-   * @brief      Declare a slice free for reuse; thread-safe
-   *
-   * @param      slice index
-   *
-   * @return     void
-   */
-  void slice_free(size_t slice_index) override
-  {
-    // Check if a slice was acually in use before and if it was, only
-    // notify the transpose threads that a free slice is available
-    bool freed = false, set_writable = false;
-    int i_buffer = 0;
-    {
-      std::unique_lock<std::mutex> lock {m_slice_mut};
-      if (!m_slice_free[slice_index]) {
-        m_slice_free[slice_index] = true;
-        freed = true;
-
-        // Clear relation between slice and buffer
-        i_buffer = std::get<0>(m_slice_to_buffer[slice_index]);
-        auto& status = m_buffer_status[i_buffer];
-        m_slice_to_buffer[slice_index] = {-1, 0, 0};
-
-        // If MEPs are not transposed and the respective buffer is no
-        // longer in use, set it to writable
-        if (
-          status.work_counter == 0 &&
-          (std::find_if(m_slice_to_buffer.begin(), m_slice_to_buffer.end(), [i_buffer](const auto& entry) {
-             return std::get<0>(entry) == i_buffer;
-           }) == m_slice_to_buffer.end())) {
-          status.writable = true;
-          set_writable = true;
-        }
-      }
-    }
-    if (freed) {
-      this->debug_output("Freed slice " + std::to_string(slice_index));
-      m_slice_cond.notify_one();
-    }
-    if (set_writable) {
-      this->debug_output("Set buffer " + std::to_string(i_buffer) + " writable");
-      m_mpi_cond.notify_one();
-    }
-  }
-
-  void event_sizes(
-    size_t const slice_index,
-    gsl::span<unsigned int const> const selected_events,
-    std::vector<size_t>& sizes) const override
-  {
-    int i_buffer = 0;
-    size_t interval_start = 0, interval_end = 0;
-    std::tie(i_buffer, interval_start, interval_end) = m_slice_to_buffer[slice_index];
-    auto const& blocks = std::get<2>(m_net_slices[i_buffer]);
-    for (unsigned int i = 0; i < selected_events.size(); ++i) {
-      auto event = selected_events[i];
-      sizes[i] +=
-        std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto& entry) {
-          auto const& block_header = std::get<0>(entry);
-          return s + bank_header_size + block_header.sizes[interval_start + event];
-        });
-    }
-  }
-
-  void copy_banks(size_t const slice_index, unsigned int const event, gsl::span<char> buffer) const override
-  {
-    auto [i_buffer, interval_start, interval_end] = m_slice_to_buffer[slice_index];
-    const auto mep_event = interval_start + event;
-
-    auto const& [mep_header, mpi_slice, blocks, fragment_offsets, slice_size] = m_net_slices[i_buffer];
-
-    unsigned char prev_type = 0;
-    auto block_index = 0;
-    size_t offset = 0;
-
-    for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
-      auto const& [block_header, block_data] = blocks[i_block];
-      auto lhcb_type = block_header.types[0];
-
-      if (prev_type != lhcb_type) {
-        block_index = 0;
-        prev_type = lhcb_type;
-      }
-
-      // All banks are taken directly from the block data to be able
-      // to treat banks needed by Allen and banks not needed by Allen
-      // in the same way
-      auto const fragment_offset = fragment_offsets[i_block][mep_event];
-      auto fragment_size = block_header.sizes[mep_event];
-
-      assert((offset + fragment_size) < static_cast<size_t>(buffer.size()));
-      offset += add_raw_bank(
-        block_header.types[mep_event],
-        mep_header.versions[i_block],
-        mep_header.source_ids[i_block],
-        {block_data.data() + fragment_offset, fragment_size},
-        buffer.data() + offset);
-      ++block_index;
-    }
-  }
-
-  int start() override
-  {
-    if (!m_started) {
-      std::unique_lock<std::mutex> lock {m_control_mutex};
-      this->debug_output("Starting", 0);
-      m_started = true;
-      m_stopping = false;
-    }
-    m_control_cond.notify_one();
-    return true;
-  };
-
-  int stop() override
-  {
-    {
-      std::unique_lock<std::mutex> lock {m_control_mutex};
-      m_stopping = true;
-      m_started = false;
-    }
-    // Make sure all threads wait for start in case they were waiting
-    // for a buffer
-    m_mpi_cond.notify_all();
-
-    return true;
-  };
-
-private:
-  void init_mpi()
-  {
-#ifdef HAVE_MPI
-
-    auto const& receivers = m_config.receivers;
-    m_domains.reserve(receivers.size());
-
-#ifdef HAVE_HWLOC
-    // Allocate and initialize topology object.
-    hwloc_topology_init(&m_topology);
-
-    // discover everything, in particular I/O devices like
-    // InfiniBand cards
-#if HWLOC_API_VERSION >= 0x20000
-    hwloc_topology_set_io_types_filter(m_topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT);
-#else
-    hwloc_topology_set_flags(m_topology, HWLOC_TOPOLOGY_FLAG_WHOLE_SYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES);
-#endif
-    // Perform the topology detection.
-    hwloc_topology_load(m_topology);
-
-    hwloc_obj_t osdev = nullptr;
-
-    if (!receivers.empty()) {
-      // Find NUMA domain of receivers
-      while ((osdev = hwloc_get_next_osdev(m_topology, osdev))) {
-        // We're interested in InfiniBand cards
-        if (osdev->attr->osdev.type == HWLOC_OBJ_OSDEV_OPENFABRICS) {
-          auto parent = hwloc_get_non_io_ancestor_obj(m_topology, osdev);
-          auto it = receivers.find(osdev->name);
-          if (it != receivers.end()) {
-            m_domains.emplace_back(it->second, parent->os_index);
-            this->debug_output(
-              "Located receiver device "s + it->first + " in NUMA domain " + std::to_string(parent->os_index));
-          }
-        }
-      }
-      if (m_domains.size() != receivers.size()) {
-        throw StrException {"Failed to locate some receiver devices "};
-      }
-    }
-#else
-    if (!receivers.empty()) {
-      info_cout << "hwloc is not available, assuming NUMA domain 0 for all receivers.\n";
-      for (auto [rec, rank] : receivers) {
-        m_domains.emplace_back(rank, 0);
-      }
-#endif
-  }
-  else { throw StrException {"MPI requested, but no receivers specified"}; }
-
-#ifdef HAVE_HWLOC
-  // Get last node. There's always at least one.
-  [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
-  assert(static_cast<size_t>(n_numa) == m_domains.size());
-
-  std::vector<hwloc_obj_t> numa_objs(m_config.n_receivers());
-  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
-    int numa_node = std::get<1>(m_domains[receiver]);
-    numa_objs[receiver] = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
-  }
-#endif
-
-  std::vector<size_t> packing_factors(m_config.n_receivers());
-  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
-    auto const receiver_rank = std::get<0>(m_domains[receiver]);
-    MPI_Recv(
-      &packing_factors[receiver],
-      1,
-      MPI_SIZE_T,
-      receiver_rank,
-      MPI::message::packing_factor,
-      MPI_COMM_WORLD,
-      MPI_STATUS_IGNORE);
-  }
-
-  if (!std::all_of(packing_factors.begin(), packing_factors.end(), [v = packing_factors.back()](auto const p) {
-        return p == v;
-      })) {
-    throw StrException {"All packing factors must be the same"};
-  }
-  else {
-    m_packing_factor = packing_factors.back();
-  }
-
-  // Allocate as many net slices as configured, of expected size
-  // Packing factor can be done dynamically if needed
-  size_t n_bytes = std::lround(m_packing_factor * average_event_size * bank_size_fudge_factor * kB);
-  for (size_t i = 0; i < m_config.n_buffers; ++i) {
-    char* contents = nullptr;
-    MPI_Alloc_mem(n_bytes, MPI_INFO_NULL, &contents);
-
-    // Only bind explicitly if there are multiple receivers,
-    // otherwise assume a memory allocation policy is in effect
-#ifdef HAVE_HWLOC
-    if (m_domains.size() > 1) {
-      auto const& numa_obj = numa_objs[numa_node];
-      auto s = hwloc_set_area_membind(
-        m_topology, contents, n_bytes, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
-      if (s != 0) {
-        throw StrException {"Failed to bind memory to node "s + std::to_string(numa_obj->os_index) + " " +
-                            strerror(errno)};
-      }
-    }
-#endif
-    cudaCheck(cudaHostRegister(contents, n_bytes, cudaHostRegisterDefault));
-    m_net_slices.emplace_back(
-      EB::Header {},
-      gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
-      MEP::Blocks {},
-      MEP::SourceOffsets {},
-      n_bytes);
-    m_mpi_buffers.emplace_back(contents);
-  }
-#else
-    error_cout << "MPI requested, but Allen was not built with MPI support.\n";
-    throw std::runtime_error {"No MPI supoprt"};
-#endif
-}
-
-size_t
-count_writable() const
-{
-  return std::accumulate(m_buffer_status.begin(), m_buffer_status.end(), 0ul, [](size_t s, BufferStatus const& stat) {
-    return s + stat.writable;
-  });
-}
-
-void allocate_storage(size_t i_read)
-{
-  if (m_sizes_known) return;
-
-  // Count number of banks per flavour
-  bool count_success = false;
-
-  // Offsets are to the start of the event, which includes the header
-  auto& [mep_header, mpi_slice, blocks, input_offsets, slice_size] = m_net_slices[i_read];
-  size_t n_blocks = mep_header.n_blocks;
-  // gsl::span<char const> block_span{mpi_slice.data() + mep_header.header_size(mep_header.n_blocks),
-  // mep_header.mep_size};
-  if (m_packing_factor == 0) {
-    m_packing_factor = mep_header.packing_factor;
-  }
-  else {
-    assert(mep_header.packing_factor == m_packing_factor);
-  }
-
-  auto eps = this->events_per_slice();
-  auto n_interval = m_packing_factor / eps;
-  auto rest = m_packing_factor % eps;
-  for (auto& s : m_buffer_status) {
-    s.intervals.reserve(2 * (n_interval + rest));
-  }
-
-  for (auto& [mep_header, mpi_slice, blocks, input_offsets, slice_size] : m_net_slices) {
-    // The number of blocks in a MEP is known, use it to allocate
-    // temporary storage used during transposition
-    blocks.resize(n_blocks);
-    input_offsets.resize(n_blocks);
-    for (auto& offsets : input_offsets) {
-      // info_cout << "Packing factor: " << mep_header.packing_factor << "\n";
-      offsets.resize(m_packing_factor + 1);
-    }
-  }
-
-  std::tie(count_success, m_banks_count) = MEP::fill_counts(mep_header, mpi_slice);
-
-  // Allocate slice memory that will contain transposed banks ready
-  // for processing by the Allen kernels
-  auto size_fun = [this, eps](BankTypes bank_type) -> std::tuple<size_t, size_t> {
-    auto it = BankSizes.find(bank_type);
-    auto ib = to_integral<BankTypes>(bank_type);
-    if (it == end(BankSizes)) {
-      throw std::out_of_range {std::string {"Bank type "} + std::to_string(ib) + " has no known size"};
-    }
-    // In case of direct MEP output, no memory should be allocated.
-    if (!m_config.transpose_mep) {
-      auto it = std::find(m_bank_ids.begin(), m_bank_ids.end(), to_integral(bank_type));
-      auto lhcb_type = std::distance(m_bank_ids.begin(), it);
-      auto n_blocks = m_banks_count[lhcb_type];
-      // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
-      return {0, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
-    }
-    else {
-      auto aps = eps < 100 ? 100 : eps;
-      return {std::lround(it->second * aps * bank_size_fudge_factor * kB), eps};
-    }
-  };
-  m_slices = allocate_slices<Banks...>(this->n_slices(), size_fun);
-  m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(this->n_slices(), std::make_tuple(-1, 0ul, 0ul));
-
-  if (!count_success) {
-    error_cout << "Failed to determine bank counts\n";
-    m_read_error = true;
-  }
-  else {
-    m_sizes_known = true;
-  }
-}
-
-/**
- * @brief      Open an input file; called from the prefetch thread
- *
- * @return     success
- */
-bool open_file() const
-{
-  bool good = false;
-
-  // Check if there are still files available
-  while (!good) {
-    // If looping on input is configured, do it
-    if (m_current == m_connections.end()) {
-      if (m_config.non_stop) {
-        m_current = m_connections.begin();
-      }
-      else {
-        break;
-      }
-    }
-
-    if (m_input) m_input->close();
-
-    m_input = MDF::open(*m_current, O_RDONLY);
-    if (m_input->good) {
-      info_cout << "Opened " << *m_current << "\n";
-      good = true;
-    }
-    else {
-      error_cout << "Failed to open " << *m_current << " " << strerror(errno) << "\n";
-      m_read_error = true;
-      return false;
-    }
-    ++m_current;
-  }
-  return good;
-}
-
-std::tuple<std::vector<BufferStatus>::iterator, size_t> get_mep_buffer(
-  std::function<bool(BufferStatus const&)> pred,
-  std::vector<BufferStatus>::iterator start,
-  std::unique_lock<std::mutex>& lock)
-{
-  // Obtain a prefetch buffer to read into, if none is available,
-  // wait until one of the transpose threads is done with its
-  // prefetch buffer
-  auto find_buffer = [this, start, &pred] {
-    auto it = std::find_if(start, m_buffer_status.end(), pred);
-    if (it == m_buffer_status.end()) {
-      it = std::find_if(m_buffer_status.begin(), start, pred);
-      if (it == start) it = m_buffer_status.end();
-    }
-    return it;
-  };
-
-  auto it = find_buffer();
-  if (it == m_buffer_status.end() && !m_transpose_done) {
-    m_mpi_cond.wait(lock, [this, &it, &find_buffer] {
-      it = find_buffer();
-      return it != m_buffer_status.end() || m_transpose_done || m_stopping;
-    });
-  }
-  return {it, distance(m_buffer_status.begin(), it)};
-}
-
-void set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events)
-{
-  if (n_events == 0) return;
-  const auto eps = this->events_per_slice();
-  auto n_interval = n_events / eps;
-  auto rest = n_events % eps;
-  if (rest) {
-    debug_cout << "Set interval (rest): " << n_interval * eps << "," << n_interval * eps + rest << "\n";
-    intervals.emplace_back(n_interval * eps, n_interval * eps + rest);
-  }
-  for (size_t i = n_interval; i != 0; --i) {
-    debug_cout << "Set interval: " << (i - 1) * eps << "," << i * eps << "\n";
-    intervals.emplace_back((i - 1) * eps, i * eps);
-  }
-}
-
-// mep reader thread
-void mep_read()
-{
-  bool receive_done = false;
-  EB::Header mep_header;
-
-  auto to_read = this->n_events();
-  if (to_read) debug_cout << "Reading " << *to_read << " events\n";
-  auto to_publish = 0;
-
-  while (!receive_done) {
-    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
-
-    // If we've been stopped, wait for start or exit
-    if (!m_started || m_stopping) {
-      std::unique_lock<std::mutex> lock {m_control_mutex};
-      this->debug_output("Waiting for start", 0);
-      m_control_cond.wait(lock, [this] { return m_started || m_done; });
-    }
-
-    if (m_done) break;
-
-    // open the first file
-    if (!m_input && !open_file()) {
-      m_read_error = true;
-      m_mpi_cond.notify_one();
-      return;
-    }
-    size_t i_buffer;
-    {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
-      std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
-      if (m_buffer_reading != m_buffer_status.end()) {
-        m_buffer_reading->writable = false;
-        assert(m_buffer_reading->work_counter == 0);
-      }
-      else {
-        continue;
-      }
-    }
-    if (m_done) {
-      receive_done = true;
-      break;
-    }
-
-    this->debug_output("Writing to MEP slice index " + std::to_string(i_buffer));
-
-    auto& read_buffer = m_read_buffers[i_buffer];
-    auto& [mep_header, buffer_span, blocks, input_offsets, buffer_size] = m_net_slices[i_buffer];
-
-    bool success = false, eof = false;
-
-    while (!success || eof) {
-      std::tie(eof, success, mep_header, buffer_span) = MEP::read_mep(*m_input, read_buffer);
-
-      if (!eof) {
-        debug_cout << "Read mep with packing factor " << mep_header.packing_factor << "\n";
-        if (to_read && success) {
-          to_publish = std::min(*to_read, size_t {mep_header.packing_factor});
-          *to_read -= to_publish;
-        }
-        else {
-          to_publish = mep_header.packing_factor;
-        }
-      }
-
-      if (!success) {
-        // Error encountered
-        m_read_error = true;
-        break;
-      }
-      else if ((to_read && *to_read == 0) || (eof && !open_file())) {
-        // Try to open the next file, if there is none, prefetching
-        // is done.
-        if (!m_read_error) {
-          this->debug_output("Prefetch done: eof and no more files");
-        }
-        receive_done = true;
-        break;
-      }
-    }
-
-    if (!m_sizes_known) {
-      allocate_storage(i_buffer);
-    }
-
-    // Notify a transpose thread that a new buffer of events is
-    // ready. If prefetching is done, wake up all threads
-    if (success) {
-      {
-        std::unique_lock<std::mutex> lock {m_mpi_mutex};
-
-        auto& status = m_buffer_status[i_buffer];
-        assert(status.work_counter == 0);
-
-        if (!eof && to_publish != 0) {
-          set_intervals(status.intervals, to_read ? to_publish : size_t {mep_header.packing_factor});
-        }
-        else {
-          // We didn't read anything, so free the buffer we got again
-          status.writable = true;
-        }
-      }
-      if (receive_done) {
-        m_done = receive_done;
-        this->debug_output("Prefetch notifying all");
-        m_mpi_cond.notify_all();
-      }
-      else if (!eof) {
-        this->debug_output("Prefetch notifying one");
-        m_mpi_cond.notify_one();
-      }
-    }
-    m_mpi_cond.notify_one();
-  }
-}
-
-#ifdef HAVE_MPI
-// MPI reader thread
-void mpi_read()
-{
-
-  int window_size = m_config.window_size;
-  std::vector<MPI_Request> requests(window_size);
-
-  // Iterate over the slices
-  size_t reporting_period = 5;
-  std::vector<std::tuple<size_t, size_t>> data_received(m_config.n_receivers());
-  std::vector<size_t> n_meps(m_config.n_receivers());
-  Timer t;
-  Timer t_origin;
-  bool error = false;
-
-  for (size_t i = 0; i < m_config.n_receivers(); ++i) {
-    auto [mpi_rank, numa_domain] = m_domains[i];
-    MPI_Recv(&n_meps[i], 1, MPI_SIZE_T, mpi_rank, MPI::message::number_of_meps, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
-  }
-  size_t number_of_meps = std::accumulate(n_meps.begin(), n_meps.end(), 0u);
-
-  size_t current_mep = 0;
-  while (!m_done && (m_config.non_stop || current_mep < number_of_meps)) {
-    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
-
-    // If we've been stopped, wait for start or exit
-    if (!m_started || m_stopping) {
-      std::unique_lock<std::mutex> lock {m_control_mutex};
-      this->debug_output("Waiting for start", 0);
-      m_control_cond.wait(lock, [this] { return m_started || m_done; });
-    }
-
-    if (m_done) break;
-
-    // Obtain a prefetch buffer to read into, if none is available,
-    // wait until one of the transpose threads is done with its
-    // prefetch buffer
-    size_t i_buffer;
-    {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
-      std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
-      if (m_buffer_reading != m_buffer_status.end()) {
-        m_buffer_reading->writable = false;
-        assert(m_buffer_reading->work_counter == 0);
-      }
-      else {
-        continue;
-      }
-    }
-
-    auto receiver = i_buffer % m_config.n_receivers();
-    auto [sender_rank, numa_node] = m_domains[receiver];
-
-    this->debug_output(
-      "Receiving from rank " + std::to_string(sender_rank) + " into buffer " + std::to_string(i_buffer) +
-      "  NUMA domain " + std::to_string(numa_node));
-
-    auto& [mep_header, buffer_span, blocks, input_offsets, buffer_size] = m_net_slices[i_buffer];
-    char*& contents = m_mpi_buffers[i_buffer];
-
-    size_t mep_size = 0;
-    MPI_Recv(&mep_size, 1, MPI_SIZE_T, sender_rank, MPI::message::event_size, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
-
-    // Reallocate if needed
-    if (mep_size > buffer_size) {
-      buffer_size = mep_size * bank_size_fudge_factor;
-      // Unregister memory
-      cudaCheck(cudaHostUnregister(contents));
-
-      // Free memory
-      MPI_Free_mem(contents);
-
-      // Allocate new memory
-      MPI_Alloc_mem(buffer_size, MPI_INFO_NULL, &contents);
-
-      // Only bind explicitly if there are multiple receivers,
-      // otherwise assume a memory allocation policy is in effect
-#ifdef HAVE_HWLOC
-      if (m_domains.size() > 1) {
-        // Bind memory to numa domain of receiving card
-        auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
-        auto s = hwloc_set_area_membind(
-          m_topology, contents, buffer_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
-        if (s != 0) {
-          m_read_error = true;
-          error_cout << "Failed to bind memory to node " << std::to_string(numa_node) << " " << strerror(errno) << "\n";
-          break;
-        }
-      }
-#endif
-      // Register memory with CUDA
-      try {
-        cudaCheck(cudaHostRegister(contents, buffer_size, cudaHostRegisterDefault));
-      } catch (std::invalid_argument const&) {
-        m_read_error = true;
-        break;
-      }
-
-      buffer_span = gsl::span {contents, static_cast<events_size>(buffer_size)};
-    }
-
-    // Number of full-size (MPI::mdf_chunk_size) messages
-    int n_messages = mep_size / MPI::mdf_chunk_size;
-    // Size of the last message (if the MFP size is not a multiple of MPI::mdf_chunk_size)
-    int rest = mep_size - n_messages * MPI::mdf_chunk_size;
-    // Number of parallel sends
-    int n_sends = n_messages > window_size ? window_size : n_messages;
-
-    // info_cout << MPI::rank_str() << "n_messages " << n_messages << ", rest " << rest << ", n_sends " << n_sends <<
-    // "\n";
-
-    // Initial parallel sends
-    for (int k = 0; k < n_sends; k++) {
-      char* message = contents + k * MPI::mdf_chunk_size;
-      MPI_Irecv(
-        message,
-        MPI::mdf_chunk_size,
-        MPI_BYTE,
-        sender_rank,
-        MPI::message::event_send_tag_start + k,
-        MPI_COMM_WORLD,
-        &requests[k]);
-    }
-    // Sliding window sends
-    for (int k = n_sends; k < n_messages; k++) {
-      int r;
-      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
-      char* message = contents + k * MPI::mdf_chunk_size;
-      MPI_Irecv(
-        message,
-        MPI::mdf_chunk_size,
-        MPI_BYTE,
-        sender_rank,
-        MPI::message::event_send_tag_start + k,
-        MPI_COMM_WORLD,
-        &requests[r]);
-    }
-    // Last send (if necessary)
-    if (rest) {
-      int r;
-      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
-      char* message = contents + n_messages * MPI::mdf_chunk_size;
-      MPI_Irecv(
-        message,
-        rest,
-        MPI_BYTE,
-        sender_rank,
-        MPI::message::event_send_tag_start + n_messages,
-        MPI_COMM_WORLD,
-        &requests[r]);
-    }
-    // Wait until all chunks have been sent
-    MPI_Waitall(n_sends, requests.data(), MPI_STATUSES_IGNORE);
-
-    mep_header = EB::Header {contents};
-    buffer_span = gsl::span {contents, static_cast<events_size>(mep_size)};
-
-    if (!m_sizes_known) {
-      allocate_storage(i_buffer);
-    }
-
-    auto& [meps_received, bytes_received] = data_received[receiver];
-    bytes_received += mep_size;
-    meps_received += 1;
-    if (t.get_elapsed_time() >= reporting_period) {
-      const auto seconds = t.get_elapsed_time();
-      auto total_rate = 0.;
-      auto total_bandwidth = 0.;
-      for (size_t i_rec = 0; i_rec < m_config.n_receivers(); ++i_rec) {
-        auto& [mr, br] = data_received[i_rec];
-        auto [rec_rank, rec_node] = m_domains[i_rec];
-
-        const double rate = (double) mr / seconds;
-        const double bandwidth = ((double) (br * 8)) / (1024 * 1024 * 1024 * seconds);
-        total_rate += rate;
-        total_bandwidth += bandwidth;
-        printf(
-          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s; Domain %2i; Rank %2i\n",
-          t_origin.get_elapsed_time(),
-          seconds,
-          rate,
-          bandwidth,
-          rec_node,
-          rec_rank);
-
-        br = 0;
-        mr = 0;
-      }
-      if (m_config.n_receivers() > 1) {
-        printf(
-          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s\n",
-          t_origin.get_elapsed_time(),
-          seconds,
-          total_rate,
-          total_bandwidth);
-      }
-      t.restart();
-    }
-
-    // Notify a transpose thread that a new buffer of events is
-    // ready. If prefetching is done, wake up all threads
-    if (!error) {
-      {
-        std::unique_lock<std::mutex> lock {m_mpi_mutex};
-        set_intervals(m_buffer_status[i_buffer].intervals, size_t {mep_header.packing_factor});
-        assert(m_buffer_status[i_buffer].work_counter == 0);
-      }
-      this->debug_output("Prefetch notifying one");
-      m_mpi_cond.notify_one();
-    }
-    m_mpi_cond.notify_one();
-
-    current_mep++;
-  }
-
-  if (!m_done) {
-    m_done = true;
-    this->debug_output("Prefetch notifying all");
-    m_mpi_cond.notify_all();
-  }
-}
-#endif
-
-/**
- * @brief      Function to run in each thread transposing events
- *
- * @param      thread ID
- *
- * @return     void
- */
-void transpose(int thread_id)
-{
-
-  size_t i_buffer = 0;
-  std::tuple<size_t, size_t> interval;
-  std::optional<size_t> slice_index;
-
-  bool good = false, transpose_full = false;
-  size_t n_transposed = 0;
-
-  auto has_intervals = [](BufferStatus const& s) { return !s.intervals.empty(); };
-
-  while (!m_read_error && !m_transpose_done) {
-    // Get a buffer to read from
-    {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
-      std::tie(m_buffer_transpose, i_buffer) = get_mep_buffer(has_intervals, m_buffer_transpose, lock);
-      if (m_transpose_done) {
-        break;
-      }
-      else if (m_buffer_transpose == m_buffer_status.end()) {
-        continue;
-      }
-      auto& status = *m_buffer_transpose;
-      assert(!status.intervals.empty());
-
-      interval = status.intervals.back();
-      status.intervals.pop_back();
-
-      ++(status.work_counter);
-      status.writable = false;
-
-      this->debug_output(
-        "Got MEP slice index " + std::to_string(i_buffer) + " interval [" + std::to_string(std::get<0>(interval)) +
-          "," + std::to_string(std::get<1>(interval)) + ")",
-        thread_id);
-    }
-
-    // Get a slice to write to
-    if (!slice_index) {
-      this->debug_output("Getting slice index", thread_id);
-      auto it = m_slice_free.end();
-      {
-        std::unique_lock<std::mutex> lock {m_slice_mut};
-        it = find(m_slice_free.begin(), m_slice_free.end(), true);
-        if (it == m_slice_free.end()) {
-          this->debug_output("Waiting for free slice", thread_id);
-          m_slice_cond.wait(lock, [this, &it] {
-            it = std::find(m_slice_free.begin(), m_slice_free.end(), true);
-            return it != m_slice_free.end() || m_transpose_done;
-          });
-          // If transpose is done and there is no slice, we were
-          // woken up be the desctructor before a slice was declared
-          // free. In that case, exit without transposing
-          if (m_transpose_done && it == m_slice_free.end()) {
-            break;
-          }
-        }
-        *it = false;
-        slice_index = distance(m_slice_free.begin(), it);
-        this->debug_output("Got slice index " + std::to_string(*slice_index), thread_id);
-
-        // Keep track of what buffer this slice belonged to
-        m_slice_to_buffer[*slice_index] = {i_buffer, std::get<0>(interval), std::get<1>(interval)};
-      }
-    }
-
-    // Reset the slice
-    auto& event_ids = m_event_ids[*slice_index];
-    reset_slice<Banks...>(m_slices, *slice_index, event_ids, !m_config.transpose_mep);
-
-    // MEP data
-    auto& [mep_header, mep_data, blocks, source_offsets, slice_size] = m_net_slices[i_buffer];
-
-    // Fill blocks
-    MEP::find_blocks(mep_header, mep_data, blocks);
-
-    // Fill fragment offsets
-    MEP::fragment_offsets(blocks, source_offsets);
-
-    // Transpose or calculate offsets
-    if (m_config.transpose_mep) {
-      // Transpose the events into the slice
-      std::tie(good, transpose_full, n_transposed) = MEP::transpose_events(
-        m_slices,
-        *slice_index,
-        m_bank_ids,
-        this->types(),
-        m_banks_count,
-        event_ids,
-        mep_header,
-        blocks,
-        source_offsets,
-        interval);
-      this->debug_output(
-        "Transposed slice " + std::to_string(*slice_index) + "; good: " + std::to_string(good) +
-          "; full: " + std::to_string(transpose_full) + "; n_transposed:  " + std::to_string(n_transposed),
-        thread_id);
-    }
-    else {
-      // Calculate fragment offsets in MEP per sub-detector
-      std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
-        m_slices, *slice_index, m_bank_ids, this->types(), m_banks_count, event_ids, mep_header, blocks, interval);
-      this->debug_output("Calculated MEP offsets for slice " + std::to_string(*slice_index), thread_id);
-    }
-
-    if (m_read_error || !good) {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
-      auto& status = m_buffer_status[i_buffer];
-      --status.work_counter;
-      m_read_error = true;
-      m_transpose_cond.notify_one();
-      break;
-    }
-
-    // Notify any threads waiting in get_slice that a slice is available
-    {
-      std::unique_lock<std::mutex> lock {m_transpose_mut};
-      m_transposed.emplace_back(*slice_index, n_transposed);
-    }
-    m_transpose_cond.notify_one();
-    slice_index.reset();
-
-    // Check if the read buffer is now empty. If it is, it can be
-    // reused, otherwise give it to another transpose thread once a
-    // new target slice is available
-    {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
-
-      auto& status = m_buffer_status[i_buffer];
-      --status.work_counter;
-
-      if (n_transposed != std::get<1>(interval) - std::get<0>(interval)) {
-        status.intervals.emplace_back(std::get<0>(interval) + n_transposed, std::get<1>(interval));
-      }
-      else if (status.work_counter == 0) {
-        m_transpose_done =
-          m_done && std::all_of(m_buffer_status.begin(), m_buffer_status.end(), [](BufferStatus const& stat) {
-            return stat.intervals.empty() && stat.work_counter == 0;
-          });
-      }
-    }
-  }
-}
-
-// Slices
-size_t m_packing_factor = 0;
-std::vector<std::vector<char>> m_read_buffers;
-std::vector<char*> m_mpi_buffers;
-MEP::Slices m_net_slices;
-
-// data members for mpi thread
-bool m_started = false;
-bool m_stopping = false;
-std::mutex m_control_mutex;
-std::condition_variable m_control_cond;
-
-// data members for mpi thread
-std::mutex m_mpi_mutex;
-std::condition_variable m_mpi_cond;
-
-#ifdef HAVE_MPI
-std::vector<std::tuple<int, int>> m_domains;
-#endif
-
-#ifdef HAVE_HWLOC
-hwloc_topology_t m_topology;
-#endif
-
-std::vector<BufferStatus> m_buffer_status;
-std::vector<BufferStatus>::iterator m_buffer_transpose;
-std::vector<BufferStatus>::iterator m_buffer_reading;
-std::thread m_input_thread;
-
-// Atomics to flag errors and completion
-std::atomic<bool> m_done = false;
-mutable std::atomic<bool> m_read_error = false;
-std::atomic<bool> m_transpose_done = false;
-
-// Buffer to store data read from file if banks are compressed. The
-// decompressed data will be written to the buffers
-mutable std::vector<char> m_compress_buffer;
-
-// Storage to read the header into for each event
-mutable LHCb::MDFHeader m_header;
-
-// Allen IDs of LHCb raw banks
-std::vector<int> m_bank_ids;
-
-// Memory slices, N for each raw bank type
-Slices m_slices;
-std::vector<std::tuple<int, size_t, size_t>> m_slice_to_buffer;
-
-// Mutex, condition varaible and queue for parallel transposition of slices
-std::mutex m_transpose_mut;
-std::condition_variable m_transpose_cond;
-std::deque<std::tuple<size_t, size_t>> m_transposed;
-
-// Keep track of what slices are free
-std::mutex m_slice_mut;
-std::condition_variable m_slice_cond;
-std::vector<bool> m_slice_free;
-
-// Threads transposing data
-std::vector<std::thread> m_transpose_threads;
-
-// Array to store the number of banks per bank type
-mutable std::array<unsigned int, LHCb::NBankTypes> m_banks_count;
-mutable bool m_sizes_known = false;
-
-// Run and event numbers present in each slice
-std::vector<EventIDs> m_event_ids;
-
-// File names to read
-std::vector<std::string> m_connections;
-
-// Storage for the currently open input file
-mutable std::optional<Allen::IO> m_input;
-
-// Iterator that points to the filename of the currently open file
-mutable std::vector<std::string>::const_iterator m_current;
-
-// Input data loop counter
-mutable size_t m_loop = 0;
-
-// Configuration struct
-MEPProviderConfig m_config;
-
-using base_class = InputProvider<MEPProvider<Banks...>>;
-}
-;
-- 
GitLab


From cbc19cd73d98cc1a2adb495cff6b24ff8f329366 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Sun, 6 Jun 2021 10:22:11 +0200
Subject: [PATCH 011/120] Initial implementation of BufferManager reading

---
 AllenOnline/AllenOnline/MEPProvider.h  |  45 +++-
 AllenOnline/AllenOnline/TransposeMEP.h |  18 +-
 AllenOnline/src/AllenApplication.cpp   |  33 ++-
 AllenOnline/src/AllenConfiguration.h   |   8 +-
 AllenOnline/src/MEPProvider.cpp        | 284 +++++++++++++++++++++----
 AllenOnline/src/TransposeMEP.cpp       |   6 +-
 6 files changed, 326 insertions(+), 68 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 94d0898e4..35dee5e0d 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -21,7 +21,10 @@
 #include <fcntl.h>
 
 #include <Event/RawBank.h>
-#include <MDF/MDFHeader.h>
+
+#include <MBM/bmdef.h>
+#include <RTL/Logger.h>
+
 #include <MDF/StreamDescriptor.h>
 
 #include "TransposeMEP.h"
@@ -37,6 +40,10 @@ namespace {
   constexpr auto bank_header_size = sizeof(LHCb::RawBank) - sizeof(unsigned int);
 } // namespace
 
+enum class MEPProviderSource {
+  Files, MBM, MPI
+};
+
 /**
  * @brief      Configuration parameters for the MEPProvider
  */
@@ -53,7 +60,7 @@ struct MEPProviderConfig {
   int window_size = 4;
 
   // Use MPI and number of receivers
-  bool mpi = false;
+  MEPProviderSource source = MEPProviderSource::MBM;
 
   bool non_stop = true;
 
@@ -65,10 +72,20 @@ struct MEPProviderConfig {
 
   // Mapping of receiver card to MPI rank to receive from
   std::map<std::string, int> receivers;
+
+  std::string bufferName = "Events";
+
+  std::string partition;
+
+  unsigned partitionID = 0;
+
+  bool partitionBuffers = false;
+
+  std::vector<std::string> requests;
 };
 
 /**
- * @brief      Provide transposed events from MDF files
+ * @brief      Provide events from MEP files in either MEP or transposed layout
  *
  * @details    The provider has three main components
  *             - a prefetch thread to read from the current input
@@ -92,9 +109,10 @@ struct MEPProviderConfig {
 class MEPProvider final : public InputProvider {
 public:
   MEPProvider(
+    std::unique_ptr<RTL::Logger>&& logger,
+    std::string proc_name,
     size_t n_slices,
     size_t events_per_slice,
-    std::optional<size_t> n_events,
     std::vector<std::string> connections,
     std::unordered_set<BankTypes> const& bank_types,
     MEPProviderConfig config = MEPProviderConfig {}) noexcept(false);
@@ -152,9 +170,11 @@ public:
 private:
   void init_mpi();
 
+  int init_bm();
+
   size_t count_writable() const;
 
-  void allocate_storage(size_t i_read);
+  bool allocate_storage(size_t i_read);
 
   /**
    * @brief      Open an input file; called from the prefetch thread
@@ -176,6 +196,9 @@ private:
   // MPI reader thread
   void mpi_read();
 
+  // buffer manager reader thread
+  void bm_read();
+
   /**
    * @brief      Function to run in each thread transposing events
    *
@@ -185,6 +208,10 @@ private:
    */
   void transpose(int thread_id);
 
+  // logging
+  std::unique_ptr<RTL::Logger> m_logger;
+  std::string m_process_name;
+
   // Slices
   size_t m_packing_factor = 0;
   std::vector<std::vector<char>> m_read_buffers;
@@ -209,7 +236,7 @@ private:
   std::vector<BufferStatus> m_buffer_status;
   std::vector<BufferStatus>::iterator m_buffer_transpose;
   std::vector<BufferStatus>::iterator m_buffer_reading;
-  std::thread m_input_thread;
+  std::vector<std::thread> m_input_threads;
 
   // Atomics to flag errors and completion
   std::atomic<bool> m_done = false;
@@ -220,9 +247,6 @@ private:
   // decompressed data will be written to the buffers
   mutable std::vector<char> m_compress_buffer;
 
-  // Storage to read the header into for each event
-  mutable LHCb::MDFHeader m_header;
-
   // Allen IDs of LHCb raw banks
   std::vector<int> m_bank_ids;
 
@@ -262,6 +286,9 @@ private:
   // Iterator that points to the filename of the currently open file
   mutable std::vector<std::string>::const_iterator m_current;
 
+  // MBM variables
+  std::vector<BMID> m_bmIDs;
+
   // Configuration struct
   MEPProviderConfig m_config;
 };
diff --git a/AllenOnline/AllenOnline/TransposeMEP.h b/AllenOnline/AllenOnline/TransposeMEP.h
index 40e43fe7c..5691b4ed7 100644
--- a/AllenOnline/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/AllenOnline/TransposeMEP.h
@@ -82,7 +82,6 @@ namespace MEP {
     std::unordered_set<BankTypes> const& bank_types,
     std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
     EventIDs& event_ids,
-    MEP::MEP_header const* mep_header,
     MEP::Blocks const& blocks,
     std::tuple<size_t, size_t> const& interval,
     bool split_by_run = false);
@@ -119,7 +118,7 @@ namespace MEP {
    * @param      event ids of banks in this slice
    * @param      start of bank data for this event
    *
-   * @return     tuple of: (success, slice is full)
+   * @return     tuple of: (success, slice is full, number of events transposed)
    */
   std::tuple<bool, bool, size_t> transpose_events(
     Allen::Slices& slices,
@@ -134,4 +133,19 @@ namespace MEP {
     std::tuple<size_t, size_t> const& interval,
     bool split_by_run = false);
 
+  /**
+   * @brief      Get MFP header from MEP
+   *
+   * @param      MEP header
+   * @param      index of MFP
+   *
+   * @return     MFP header
+   */
+  inline MFP::MFP_header const* block_header(MEP::MEP_header const* mep_header, size_t block_index)
+  {
+    // Offsets to MFPs are in number of 4-byte words
+    auto const* data = reinterpret_cast<uint32_t const*>(mep_header);
+    auto const offset = mep_header->offsets()[block_index];
+    return reinterpret_cast<MFP::MFP_header const*>(data + offset);
+  }
 } // namespace MEP
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index e3d996a47..d50904149 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -166,7 +166,7 @@ int AllenApplication::configureApplication()
     return Online::ONLINE_ERROR;
   }
 
-  if (m_allenConfig->withMPI.value()) {
+  if (m_allenConfig->source.value() == "MPI") {
     auto success = initMPI();
     if (!success) {
       m_logger->error("Failed to initialize MPI");
@@ -332,7 +332,7 @@ void AllenApplication::allenLoop()
     allen_options["s"] = std::to_string(m_allenConfig->nSlices.value());
   }
 
-  if (!m_allenConfig->withMPI.value() && m_allenConfig->input.value().empty()) {
+  if (m_allenConfig->source.value() == "Files" && m_allenConfig->input.value().empty()) {
     m_logger->throwError("No input files specified");
   }
 
@@ -341,15 +341,31 @@ void AllenApplication::allenLoop()
 
 std::unique_ptr<IInputProvider> AllenApplication::makeProvider()
 {
+  std::unordered_map<std::string, MEPProviderSource> sources{{"MBM", MEPProviderSource::MBM},
+                                                             {"MPI", MEPProviderSource::MPI},
+                                                             {"Files", MEPProviderSource::Files}};
+
+  // FIXME: Do this in AllenConfiguration in the proper way with
+  // property parsing
+  if (!sources.count(m_allenConfig->source.value())) {
+    m_logger->error(std::string{"MEPProviderSource must be one of MBM, MPI, Files. Got "} + m_allenConfig->source.value());
+    return {};
+  }
+
   MEPProviderConfig config {false,                // verify MEP checksums
                             10,                   // number of read buffers
                             m_allenConfig->transpose.value() ? 4u : 1u, // number of transpose threads
                             4,                                          // MPI sliding window size
-                            m_allenConfig->withMPI.value(),             // Receive from MPI or read files
+                            sources[m_allenConfig->source.value()],     // Receive from MPI, MBM or files
                             m_allenConfig->nonStop.value(),             // Loop over input non-stop
                             m_allenConfig->transpose.value(),           // MEPs should be transposed to Allen layout
                             m_allenConfig->runChanges.value(),          // Whether to split slices by run number
-                            m_allenConfig->receivers.value()};          // Map of receiver to MPI rank to receive from
+                            m_allenConfig->receivers.value(),           // Map of receiver to MPI rank to receive from
+                            m_allenConfig->bufferName.value(),          // MBM buffer name
+                            m_allenConfig->partition.value(),           // Partition name
+                            m_allenConfig->partitionID.value(),         // Parition ID
+                            m_allenConfig->partitionBuffers.value(),    // Partition buffers using partition name
+                            m_allenConfig->requests.value()};           // MBM requests
 
   std::unordered_set<BankTypes> bankTypes;
   for (auto bt : m_allenConfig->bankTypes.value()) {
@@ -363,5 +379,12 @@ std::unique_ptr<IInputProvider> AllenApplication::makeProvider()
     }
   }
 
-  return std::make_unique<MEPProvider>(m_allenConfig->nSlices.value(), m_allenConfig->eps.value(), std::optional<size_t>{}, m_allenConfig->input.value(), bankTypes, config);
+  auto io_logger = m_logger->clone("MEPProvider", m_config->outputLevel());
+  return std::make_unique<MEPProvider>(std::move(io_logger),
+                                       name(),
+                                       m_allenConfig->nSlices.value(),
+                                       m_allenConfig->eps.value(),
+                                       m_allenConfig->input.value(),
+                                       bankTypes,
+                                       config);
 }
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index 0ec6a6922..ee2150dc0 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -22,7 +22,7 @@ public:
   Gaudi::Property<int> eps {this, "EventsPerSlice", 1000};
   Gaudi::Property<bool> nonStop {this, "NonStop", true};
   Gaudi::Property<bool> transpose {this, "TransposeMEP", false};
-  Gaudi::Property<bool> withMPI {this, "MPI", true};
+  Gaudi::Property<std::string> source {this, "Source", "MBM"};
   Gaudi::Property<std::map<std::string, int>> receivers {this, "Receivers"};
   Gaudi::Property<bool> check {this, "CheckMC", false};
   Gaudi::Property<unsigned int> nThreads {this, "NThreads", 8};
@@ -33,6 +33,12 @@ public:
   Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json"};
   Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
   Gaudi::Property<std::vector<std::string>> input {this, "Input"};
+  Gaudi::Property<std::string> bufferName {this, "BufferName", "Events"};
+  Gaudi::Property<std::string> partition {this, "Partition"};
+  Gaudi::Property<unsigned> partitionID {this, "PartitionID"};
+  Gaudi::Property<bool> partitionBuffers {this, "PartitionBuffers", false};
+  Gaudi::Property<std::vector<std::string>> requests {this, "Requests"};
+
   Gaudi::Property<std::unordered_set<LHCb::RawBank::BankType>> bankTypes{this, "BankTypes",
                                                                   {LHCb::RawBank::VP,
                                                                    LHCb::RawBank::UT,
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 7b6fde813..0fa3c8923 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -14,6 +14,8 @@
 
 #include <Event/RawBank.h>
 
+#include <MBM/Requirement.h>
+
 #include "AllenOnline/MEPProvider.h"
 #include "AllenOnline/TransposeMEP.h"
 
@@ -25,33 +27,49 @@
 #include <hwloc.h>
 #endif
 
-
+// The MEPProvider has three possible sources: MPI, the BufferManager
+// and files.
+//
+// In case of file input, MEPs are read into m_read_buffers and
+// reading will continue until all read buffers are full or the
+// maximum number of events has been reached. The memory in the read
+// buffers is owned by the MEPProvider.
+//
+// In case of MPI input, memory to receive MEPs is allocated with MPI
+// and the pointers are stored in m_mpi_buffers. Memory allocated with
+// MPI is registered to the device runtime.
+//
+// In case of BufferManager input, a number of input threads equal to
+// the number of buffers is started. This ensures a 1-to-1 mapping
+// between buffers and BMIDs stored in m_bmIDs. All of the memory
+// allocated by the BufferManager is registered with the device
+// runtime.
 
 MEPProvider::MEPProvider(
+  std::unique_ptr<RTL::Logger>&& logger,
+  std::string proc_name,
   size_t n_slices,
   size_t events_per_slice,
-  std::optional<size_t> n_events,
   std::vector<std::string> connections,
   std::unordered_set<BankTypes> const& bank_types,
   MEPProviderConfig config) noexcept(false) :
 InputProvider {n_slices, events_per_slice, bank_types,
-  config.transpose_mep ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP,
-  n_events},
-  m_buffer_status(config.n_buffers), m_slice_free(n_slices, true), m_banks_count {0}, m_event_ids {n_slices},
-  m_connections {std::move(connections)}, m_config {config}
+  config.transpose_mep ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, {}},
+  m_logger(std::move(logger)), m_process_name{std::move(proc_name)}, m_buffer_status(config.n_buffers), m_slice_free(n_slices, true),
+  m_banks_count {0}, m_event_ids {n_slices}, m_connections {std::move(connections)}, m_config {config}
 {
 
   if (m_config.transpose_mep) {
-    info_cout << "Providing events in Allen layout by transposing MEPs\n";
+    m_logger->info("Providing events in Allen layout by transposing MEPs");
   }
   else {
-    info_cout << "Providing events in MEP layout\n";
+    m_logger->info("Providing events in MEP layout");
   }
 
   m_buffer_transpose = m_buffer_status.begin();
   m_buffer_reading = m_buffer_status.begin();
 
-  if (m_config.mpi) {
+  if (m_config.source == MEPProviderSource::MPI) {
     init_mpi();
   }
   else {
@@ -73,28 +91,16 @@ InputProvider {n_slices, events_per_slice, bank_types,
   // Reserve 1MB for decompression
   m_compress_buffer.reserve(1u * MB);
 
-  // start MPI receive or MEP reading thread
-  if (m_config.mpi) {
-  #ifdef HAVE_MPI
-    m_input_thread = std::thread {&MEPProvider::mpi_read, this};
-  #else
-    throw StrException {"MPI requested, but no MPI support built in."};
-  #endif
-  }
-  else {
-    m_input_thread = std::thread {&MEPProvider::mep_read, this};
-  }
-
   // Sanity check on the number of buffers and threads
   if (m_config.n_buffers <= 1) {
-    warning_cout << "too few read buffers requested, setting it to 2\n";
+    m_logger->warning("too few read buffers requested, setting it to 2");
     m_config.n_buffers = 2;
   }
 
   if (m_config.n_transpose_threads > m_config.n_buffers - 1) {
-    warning_cout << "too many transpose threads requested with respect "
-                    "to the number of read buffers; reducing the number of threads to "
-                 << m_config.n_buffers - 1;
+    m_logger->warning(std::string{"too many transpose threads requested with respect "} +
+                      "to the number of read buffers; reducing the number of threads to "
+                      + std::to_string(m_config.n_buffers - 1));
     m_config.n_transpose_threads = m_config.n_buffers - 1;
   }
 
@@ -207,6 +213,9 @@ void MEPProvider::slice_free(size_t slice_index)
          }) == m_slice_to_buffer.end())) {
         status.writable = true;
         set_writable = true;
+        if (m_config.source == MEPProviderSource::MBM) {
+          ::mbm_free_event(m_bmIDs[i_buffer]);
+        }
       }
     }
   }
@@ -278,14 +287,38 @@ void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event,
 
 int MEPProvider::start()
 {
-  if (!m_started) {
+  if (m_started) {
+    return true;
+  }
+  else {
+    // start MPI receive, MEP reading thread or BM thread
+    if (m_config.source == MEPProviderSource::MPI) {
+      #ifdef HAVE_MPI
+      m_input_threads.emplace_back(&MEPProvider::mpi_read, this);
+      #else
+      throw StrException {"MPI requested, but no MPI support built in."};
+      #endif
+    }
+    else if (m_config.source == MEPProviderSource::Files) {
+      m_input_threads.emplace_back(&MEPProvider::mep_read, this);
+    } else {
+      auto sc = init_bm();
+      if (sc != MBM_NORMAL) {
+        return false;
+      }
+      for (size_t b = 0; b < m_config.n_buffers; ++b) {
+        m_input_threads.emplace_back(&MEPProvider::bm_read, this);
+      }
+    }
+
     std::unique_lock<std::mutex> lock {m_control_mutex};
     this->debug_output("Starting", 0);
+
     m_started = true;
     m_stopping = false;
+    m_control_cond.notify_one();
+    return true;
   }
-  m_control_cond.notify_one();
-  return true;
 };
 
 int MEPProvider::stop()
@@ -295,6 +328,15 @@ int MEPProvider::stop()
     m_stopping = true;
     m_started = false;
   }
+
+  if (m_config.source == MEPProviderSource::MBM) {
+    // Cancel all requests to the buffer manager for those who are waiting
+    std::unique_lock<std::mutex> lock {m_mpi_mutex};
+    for (size_t b = 0; b < m_buffer_status.size(); ++b) {
+      if (!m_buffer_status[b].writable) ::mbm_cancel_request(m_bmIDs[b]);
+    }
+  }
+
   // Make sure all threads wait for start in case they were waiting
   // for a buffer
   m_mpi_cond.notify_all();
@@ -408,11 +450,60 @@ void MEPProvider::init_mpi()
     m_mpi_buffers.emplace_back(contents);
   }
   #else
-  error_cout << "MPI requested, but Allen was not built with MPI support.\n";
+  m_logger->error("MPI requested, but Allen was not built with MPI support.");
   throw std::runtime_error {"No MPI supoprt"};
   #endif
 }
 
+int MEPProvider::init_bm()
+{
+  auto buffer_name = m_config.bufferName;
+  if ( m_config.partitionBuffers ) {
+    std::stringstream stream;
+    stream << std::hex << m_config.partitionID;
+    buffer_name += "_";
+    buffer_name += m_config.partition.empty() ? stream.str() : m_config.partition;
+  }
+
+
+  for (size_t b = 0; b < m_config.n_buffers; ++b) {
+    auto pn = m_process_name + "." + std::to_string(b);
+    BMID bmid;
+    if (b == 0) {
+      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), m_config.partitionID, BM_COM_FIFO);
+    } else {
+      bmid = ::mbm_connect(m_bmIDs[0], pn.c_str(), m_config.partitionID);
+    }
+
+    if (bmid == MBM_INV_DESC)   {
+      return m_logger->error("MBM: Failed to connect to MBM buffer %s!", buffer_name.c_str());
+      return MBM_ERROR;
+    } else if (b == 0) {
+      // register buffer manager memory with the device runtime
+      char const* mem = nullptr;
+      size_t mem_size = 0;
+      auto sc = mbm_buffer_memory(bmid, &mem, &mem_size);
+      if (sc == MBM_NORMAL) {
+        Allen::host_register(const_cast<char*>(mem), mem_size, Allen::hostRegisterDefault);
+      } else {
+        m_logger->error("MBM: Failed to obtain buffer start and size, MBM");
+        return MBM_ERROR;
+      }
+    }
+    for(auto r : m_config.requests) {
+      MBM::Requirement rq{r};
+      int sc = ::mbm_add_req(bmid, rq.evtype, rq.trmask, rq.vetomask, rq.maskType,
+                             rq.userType, rq.freqType, rq.freq);
+      if (sc != MBM_NORMAL)   {
+        m_logger->error("MBM: Failed to add MBM requirement: %s",r.c_str());
+        return MBM_ERROR;
+      }
+    }
+    m_bmIDs.emplace_back(bmid);
+  }
+  return MBM_NORMAL;
+}
+
 size_t MEPProvider::count_writable() const
 {
   return std::accumulate(m_buffer_status.begin(), m_buffer_status.end(), 0ul, [](size_t s, BufferStatus const& stat) {
@@ -420,9 +511,9 @@ size_t MEPProvider::count_writable() const
   });
 }
 
-void MEPProvider::allocate_storage(size_t i_read)
+bool MEPProvider::allocate_storage(size_t i_read)
 {
-  if (m_sizes_known) return;
+  if (m_sizes_known) return true;
 
   // Count number of banks per flavour
   bool count_success = false;
@@ -485,11 +576,12 @@ void MEPProvider::allocate_storage(size_t i_read)
   m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(this->n_slices(), std::make_tuple(-1, 0ul, 0ul));
 
   if (!count_success) {
-    error_cout << "Failed to determine bank counts\n";
-    m_read_error = true;
+    m_logger->error("Failed to determine bank counts");
+    return false;
   }
   else {
     m_sizes_known = true;
+    return true;
   }
 }
 
@@ -517,7 +609,7 @@ bool MEPProvider::open_file() const
       good = true;
     }
     else {
-      error_cout << "Failed to open " << *m_current << " " << strerror(errno) << "\n";
+      m_logger->error(std::string{"Failed to open "} + *m_current + " " + strerror(errno));
       m_read_error = true;
       return false;
     }
@@ -526,9 +618,9 @@ bool MEPProvider::open_file() const
   return good;
 }
 
-std::tuple<std::vector<BufferStatus>::iterator, size_t> MEPProvider::get_mep_buffer(
-  std::function<bool(BufferStatus const&)> pred,
-  std::vector<BufferStatus>::iterator start,
+std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvider::get_mep_buffer(
+  std::function<bool(IInputProvider::BufferStatus const&)> pred,
+  std::vector<IInputProvider::BufferStatus>::iterator start,
   std::unique_lock<std::mutex>& lock)
 {
   // Obtain a prefetch buffer to read into, if none is available,
@@ -651,8 +743,9 @@ void MEPProvider::mep_read()
       }
     }
 
-    if (!m_sizes_known) {
-      allocate_storage(i_buffer);
+    if (!m_sizes_known && !allocate_storage(i_buffer)) {
+      m_read_error = true;
+      break;
     }
 
     assert(slice.packing_factor = m_packing_factor);
@@ -773,7 +866,7 @@ void MEPProvider::mpi_read()
           m_topology, contents, slice.slice_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
         if (s != 0) {
           m_read_error = true;
-          error_cout << "Failed to bind memory to node " << std::to_string(numa_node) << " " << strerror(errno) << "\n";
+          m_logger->error(std::string{"Failed to bind memory to node "} + std::to_string(numa_node) + " " + strerror(errno));
           break;
         }
       }
@@ -846,11 +939,12 @@ void MEPProvider::mpi_read()
     slice.mep_data = gsl::span {contents, static_cast<events_size>(mep_size)};
 
     auto const* mep_header = slice.mep_header;
-    auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0]);
-    slice.packing_factor = block_header->n_banks;
+    auto const* bh = MEP::block_header(mep_header, 0);
+    slice.packing_factor = bh->n_banks;
 
-    if (!m_sizes_known) {
-      allocate_storage(i_buffer);
+    if (!m_sizes_known && !allocate_storage(i_buffer)) {
+      m_read_error = true;
+      break;
     }
 
     auto& [meps_received, bytes_received] = data_received[receiver];
@@ -916,6 +1010,104 @@ void MEPProvider::mpi_read()
 }
 
 
+// buffer manager reader thread
+void MEPProvider::bm_read()
+{
+  auto to_read = this->n_events();
+  if (to_read)
+    m_logger->error("Number of events makes no sense when receiving from"
+                    " the buffer manager: ignoring");
+
+  while (!m_done) {
+    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
+
+    // If we've been stopped, wait for start or exit
+    if (!m_started || m_stopping) {
+      std::unique_lock<std::mutex> lock {m_control_mutex};
+      this->debug_output("Waiting for start", 0);
+      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+    }
+
+    if (m_done) break;
+
+    size_t i_buffer;
+    {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::tie(m_buffer_reading, i_buffer) =
+        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+      if (m_buffer_reading != m_buffer_status.end()) {
+        m_buffer_reading->writable = false;
+        assert(m_buffer_reading->work_counter == 0);
+      }
+      else {
+        continue;
+      }
+    }
+    if (m_done) {
+      break;
+    }
+
+    this->debug_output("Writing to MEP slice index " + std::to_string(i_buffer));
+
+    auto& slice = m_net_slices[i_buffer];
+
+    bool cancelled = false;
+
+    while (!cancelled) {
+      unsigned int trmask[BM_MASK_SIZE];
+      int ev_type = 0, *ev_data = 0;
+      long ev_len = 0;
+
+      auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, m_config.partitionID);
+      if (sc == MBM_NORMAL)  {
+        slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(ev_data);
+        slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep_header->bytes()};
+        slice.slice_size = static_cast<size_t>(ev_len);
+        auto const* block_header = MEP::block_header(slice.mep_header, 0);
+        slice.packing_factor = block_header->n_banks;
+
+        m_logger->debug(std::string{"Got mep with packing factor "} + std::to_string(slice.packing_factor));
+      } else if (sc == MBM_REQ_CANCEL) {
+        std::unique_lock<std::mutex> lock {m_mpi_mutex};
+        m_buffer_status[i_buffer].writable = true;
+        cancelled = true;
+        if (count_writable() == 0) {
+          m_done = true;
+          break;
+        }
+      }
+    }
+
+    if (!m_sizes_known && !allocate_storage(i_buffer)) {
+      m_read_error = true;
+      break;
+    }
+
+    assert(slice.packing_factor = m_packing_factor);
+
+    // Notify a transpose thread that a new buffer of events is
+    // ready. If prefetching is done, wake up all threads
+    if (!cancelled) {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+
+      auto& status = m_buffer_status[i_buffer];
+      assert(status.work_counter == 0);
+
+      set_intervals(status.intervals, size_t {slice.packing_factor});
+    }
+
+    if (m_done) {
+      this->debug_output("Prefetch notifying all");
+      m_mpi_cond.notify_all();
+    }
+    else if (!cancelled) {
+      this->debug_output("Prefetch notifying one");
+      m_mpi_cond.notify_one();
+    }
+  }
+}
+
+
 /**
  * @brief      Function to run in each thread transposing events
  *
@@ -995,8 +1187,6 @@ void MEPProvider::transpose(int thread_id)
     //    reset_slice(m_slices, *slice_index, bank_types, event_ids, !m_config.transpose_mep);
     reset_slice(m_slices, *slice_index, types(), event_ids, !m_config.transpose_mep);
 
-
-
     // MEP data
     auto& slice = m_net_slices[i_buffer];
 
@@ -1028,7 +1218,7 @@ void MEPProvider::transpose(int thread_id)
     else {
       // Calculate fragment offsets in MEP per sub-detector
       std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
-        m_slices, *slice_index, m_bank_ids, this->types(), m_banks_count, event_ids, slice.mep_header, slice.blocks, interval);
+        m_slices, *slice_index, m_bank_ids, this->types(), m_banks_count, event_ids, slice.blocks, interval);
       this->debug_output("Calculated MEP offsets for slice " + std::to_string(*slice_index), thread_id);
     }
 
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 05441d170..fb891a727 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -32,8 +32,7 @@ MEP::fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& mep
   std::array<unsigned int, LHCb::NBankTypes> count {0};
   std::array<int, NBankTypes> versions {0};
   for (size_t i = 0; i < header->n_MFPs; ++i) {
-    auto offset = header->offsets()[i];
-    auto const* bh = reinterpret_cast<MFP::MFP_header const*>(block_span.data() + offset);
+    auto const* bh = block_header(header, i);
 
     // info_cout << "EB BlockHeader: "
     //   << bh.event_id << ", " << bh.n_frag << ", " << bh.reserved << ", " << bh.block_size << "\n";
@@ -53,7 +52,6 @@ MEP::fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& mep
 
 void MEP::find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor, gsl::span<char const> const& buffer_span, Blocks& blocks)
 {
-
   // Fill blocks
   auto block_hdr_size = MFP::MFP_header_size(packing_factor, MFP::MFP_aligment);
 
@@ -61,7 +59,7 @@ void MEP::find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor
     // block offsets are in 4-byte words with respect to the start of the MEP header
     auto block_offset = mep_header->offsets()[i_block] * sizeof(uint32_t);
     MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>(buffer_span.data() + block_offset);
-    gsl::span<char const> block_data {buffer_span.data() + block_offset + block_hdr_size, block_header->bytes()};
+    gsl::span<char const> block_data {buffer_span.data() + block_offset + block_hdr_size, block_header->bytes() -  block_hdr_size};
     blocks[i_block] = std::tuple {block_header, std::move(block_data)};
   }
 }
-- 
GitLab


From efb41fe0bb9bbdd2ca4eb44e2de6740d36ff77c4 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 9 Jun 2021 20:09:49 +0200
Subject: [PATCH 012/120] Add MBMOutput and refactor MEPProvider into a service

---
 AllenOnline/AllenOnline/MBMOutput.h   |  59 ++++
 AllenOnline/AllenOnline/MEPProvider.h | 126 ++++---
 AllenOnline/CMakeLists.txt            |   7 +-
 AllenOnline/src/AllenApplication.cpp  | 205 ++++++------
 AllenOnline/src/AllenApplication.h    |  15 +-
 AllenOnline/src/AllenConfiguration.h  |  27 +-
 AllenOnline/src/MBMOutput.cpp         | 106 ++++++
 AllenOnline/src/MEPProvider.cpp       | 459 +++++++++++++++-----------
 8 files changed, 602 insertions(+), 402 deletions(-)
 create mode 100644 AllenOnline/AllenOnline/MBMOutput.h
 create mode 100644 AllenOnline/src/MBMOutput.cpp

diff --git a/AllenOnline/AllenOnline/MBMOutput.h b/AllenOnline/AllenOnline/MBMOutput.h
new file mode 100644
index 000000000..bf8a417af
--- /dev/null
+++ b/AllenOnline/AllenOnline/MBMOutput.h
@@ -0,0 +1,59 @@
+#pragma once
+
+#include <MBM/bmdef.h>
+#include <RTL/Logger.h>
+
+#include <Allen/OutputHandler.h>
+
+typedef void* BMID;
+
+class MBMOutput final : public OutputHandler {
+public:
+  MBMOutput(
+    std::unique_ptr<RTL::Logger>&& logger,
+    IInputProvider const* inputProvider,
+    std::string bufferName,
+    std::string processName,
+    unsigned partitionID,
+    size_t const nLines,
+    bool const checksum = true);
+
+  /// Callback when space is availible (not really used)
+  static int spaceCallback(void* /* param */);
+
+protected:
+
+  std::tuple<size_t, gsl::span<char>> buffer(size_t buffer_size) override;
+
+  bool write_buffer(size_t) override;
+
+  bool start() override;
+
+  bool stop() override;
+
+  void cancel() override;
+
+private:
+  // Logger
+  std::unique_ptr<RTL::Logger> m_logger;
+
+  // Output buffer name
+  std::string const m_bufferName;
+
+  // Process name
+  std::string const m_processName;
+
+  // partition ID
+  unsigned m_partitionID = 0;
+
+  // do checksum on write
+  bool const m_checksum;
+
+  // MBM buffer
+  gsl::span<char> m_buffer;
+
+  // Output buffer ID
+  BMID m_bmID = MBM_INV_DESC;
+
+  bool m_cancelled = false;
+};
diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 35dee5e0d..900240729 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -20,6 +20,11 @@
 #include <sys/stat.h>
 #include <fcntl.h>
 
+#include <Gaudi/Property.h>
+#include <Kernel/meta_enum.h>
+
+#include <GaudiKernel/Service.h>
+
 #include <Event/RawBank.h>
 
 #include <MBM/bmdef.h>
@@ -40,49 +45,17 @@ namespace {
   constexpr auto bank_header_size = sizeof(LHCb::RawBank) - sizeof(unsigned int);
 } // namespace
 
-enum class MEPProviderSource {
-  Files, MBM, MPI
-};
+namespace MEP {
+  meta_enum_class(ProviderSource, int,
+                  Unknown = 0,
+                  Files,
+                  MBM,
+                  MPI)
+}
 
 /**
  * @brief      Configuration parameters for the MEPProvider
  */
-struct MEPProviderConfig {
-  // check the MDF checksum if it is available
-  bool check_checksum = false;
-
-  // number of prefetch buffers
-  size_t n_buffers = 8;
-
-  // number of transpose threads
-  size_t n_transpose_threads = 5;
-
-  int window_size = 4;
-
-  // Use MPI and number of receivers
-  MEPProviderSource source = MEPProviderSource::MBM;
-
-  bool non_stop = true;
-
-  bool transpose_mep = false;
-
-  bool split_by_run = false;
-
-  size_t n_receivers() const { return receivers.size(); }
-
-  // Mapping of receiver card to MPI rank to receive from
-  std::map<std::string, int> receivers;
-
-  std::string bufferName = "Events";
-
-  std::string partition;
-
-  unsigned partitionID = 0;
-
-  bool partitionBuffers = false;
-
-  std::vector<std::string> requests;
-};
 
 /**
  * @brief      Provide events from MEP files in either MEP or transposed layout
@@ -106,16 +79,9 @@ struct MEPProviderConfig {
  * @param      Configuration struct
  *
  */
-class MEPProvider final : public InputProvider {
+class MEPProvider final : public Service, public InputProvider {
 public:
-  MEPProvider(
-    std::unique_ptr<RTL::Logger>&& logger,
-    std::string proc_name,
-    size_t n_slices,
-    size_t events_per_slice,
-    std::vector<std::string> connections,
-    std::unordered_set<BankTypes> const& bank_types,
-    MEPProviderConfig config = MEPProviderConfig {}) noexcept(false);
+  MEPProvider(std::string name, ISvcLocator* loc);
 
   /**
    * @brief      Obtain event IDs of events stored in a given slice
@@ -163,12 +129,15 @@ public:
 
   void copy_banks(size_t const slice_index, unsigned int const event, gsl::span<char> buffer) const override;
 
-  int start() override;
+  StatusCode initialize() override;
+
+  StatusCode start() override;
 
-  int stop() override;
+  StatusCode stop() override;
 
 private:
-  void init_mpi();
+
+  StatusCode init_mpi();
 
   int init_bm();
 
@@ -208,10 +177,6 @@ private:
    */
   void transpose(int thread_id);
 
-  // logging
-  std::unique_ptr<RTL::Logger> m_logger;
-  std::string m_process_name;
-
   // Slices
   size_t m_packing_factor = 0;
   std::vector<std::vector<char>> m_read_buffers;
@@ -231,6 +196,10 @@ private:
   #ifdef HAVE_MPI
   std::vector<std::tuple<int, int>> m_domains;
   hwloc_topology_t m_topology;
+
+  char** m_mpiArgv = nullptr;
+  int m_mpiArgc = 1;
+  int m_rank = -1;
   #endif
 
   std::vector<BufferStatus> m_buffer_status;
@@ -243,10 +212,6 @@ private:
   mutable std::atomic<bool> m_read_error = false;
   std::atomic<bool> m_transpose_done = false;
 
-  // Buffer to store data read from file if banks are compressed. The
-  // decompressed data will be written to the buffers
-  mutable std::vector<char> m_compress_buffer;
-
   // Allen IDs of LHCb raw banks
   std::vector<int> m_bank_ids;
 
@@ -277,18 +242,51 @@ private:
   // Run and event numbers present in each slice
   std::vector<EventIDs> m_event_ids;
 
-  // File names to read
-  std::vector<std::string> m_connections;
-
   // Storage for the currently open input file
   mutable std::optional<LHCb::StreamDescriptor::Access> m_input;
 
   // Iterator that points to the filename of the currently open file
   mutable std::vector<std::string>::const_iterator m_current;
 
+  size_t n_buffers() const { return std::get<0>(m_bufferConfig.value()); }
+
+  size_t n_receivers() const { return m_receivers.size(); }
+
   // MBM variables
   std::vector<BMID> m_bmIDs;
 
-  // Configuration struct
-  MEPProviderConfig m_config;
+  Gaudi::Property<std::set<LHCb::RawBank::BankType>>
+  m_lhcb_bank_types{this, "BankTypes",
+                    {LHCb::RawBank::VP,
+                     LHCb::RawBank::UT,
+                     LHCb::RawBank::FTCluster,
+                     LHCb::RawBank::Muon,
+                     LHCb::RawBank::ODIN,
+                     LHCb::RawBank::EcalPacked,
+                     LHCb::RawBank::HcalPacked}};
+
+  std::unordered_set<BankTypes> m_bank_types;
+
+  Gaudi::Property<std::string> m_process_name {this, "ProcessName"};
+  Gaudi::Property<size_t> m_nslices {this, "NSlices", 6};
+  Gaudi::Property<size_t> m_events_per_slice {this, "EventsPerSlice", 1000};
+  Gaudi::Property<std::vector<std::string>> m_connections {this, "Connections"};
+
+  Gaudi::Property<MEP::ProviderSource> m_source {this, "Source", MEP::ProviderSource::MBM};
+
+  // number of prefetch buffers and transpose threads
+  Gaudi::Property<std::pair<int, int>> m_bufferConfig {this, "BufferConfig", {8, 5}};
+  Gaudi::Property<int> m_window_size {this, "MPIWindow", 4};
+  Gaudi::Property<bool> m_non_stop {this, "LoopOnMEPs", false};
+  Gaudi::Property<bool> m_transpose_mep {this, "TransposeMEPs", false};
+  Gaudi::Property<bool> m_split_by_run {this, "SplitByRun", true};
+
+  // Mapping of receiver card to MPI rank to receive from
+  Gaudi::Property<std::map<std::string, int>> m_receivers {this, "Receivers", {}};
+  Gaudi::Property<std::string> m_bufferName {this, "BufferName", "Events"};
+  Gaudi::Property<std::string> m_partition {this, "Partition", "Test"};
+  Gaudi::Property<unsigned> m_partitionID {this, "PartitionID", 0};
+  Gaudi::Property<bool> m_partitionBuffers {this, "PartitionBuffers", false};
+  Gaudi::Property<std::vector<std::string>> m_requests {this, "Requests", {}};
+
 };
diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 2aa760c4f..364a19cc1 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -15,13 +15,11 @@ gaudi_depends_on_subdirs(Online/OnlineKernel
 find_package(cppgsl REQUIRED)
 find_package(ZMQ REQUIRED)
 find_package(Boost REQUIRED COMPONENTS program_options)
-find_package(ROOT REQUIRED)
 find_package(CUDAToolkit REQUIRED)
 
 include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
   ${ZMQ_INCLUDE_DIRS}
-  ${CPPGSL_INCLUDE_DIR}
-  ${ROOT_INCLUDE_DIRS})
+  ${CPPGSL_INCLUDE_DIR})
 
 include_directories(include)
 
@@ -48,7 +46,8 @@ gaudi_add_module(AllenOnlineComp
                  src/AllenApplication.cpp
                  src/AllenConfiguration.cpp
                  src/MEPProvider.cpp
-                 INCLUDE_DIRS OnlineBase ROOT Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
+                 src/MBMOutput.cpp
+                 INCLUDE_DIRS OnlineBase Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
                  LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQLib RPC Parsers EventBuildingLib MDFLib
                                 OnlineKernel ${ALLEN_LIBRARY} BinaryDumpers DAQEventLib
                                 DAQKernelLib GaudiAlgLib PrKernel VPDetLib UTDetLib UTKernelLib AllenOnline
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index d50904149..632813ebd 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -7,6 +7,8 @@
 #include <regex>
 #include <unordered_set>
 
+#include <filesystem>
+
 #include <GaudiKernel/IJobOptionsSvc.h>
 #include <GaudiKernel/IMessageSvc.h>
 #include <GaudiKernel/IAppMgrUI.h>
@@ -16,6 +18,7 @@
 #include <Gaudi/Property.h>
 #include <GaudiKernel/SmartIF.h>
 #include <GaudiKernel/IMonitorSvc.h>
+
 #include <CPP/Event.h>
 #include <RTL/strdef.h>
 #include <RTL/rtl.h>
@@ -26,8 +29,10 @@
 #include <Allen/BankTypes.h>
 #include <Allen/BankMapping.h>
 #include <Allen/Allen.h>
+#include <Allen/Provider.h>
 
 #include <AllenOnline/MEPProvider.h>
+#include <AllenOnline/MBMOutput.h>
 
 #ifdef HAVE_MPI
 #include <MPIConfig.h>
@@ -52,6 +57,8 @@ namespace {
     }
     return s;
   }
+
+  namespace fs = std::filesystem;
 } // namespace
 
 /// Factory instantiation
@@ -78,9 +85,12 @@ AllenApplication::~AllenApplication()
 /// Stop the application                             (RUNNING    -> READY)
 int AllenApplication::stop()
 {
-  fireIncident("DAQ_CANCEL");
+  m_zmqSvc->send(*m_allenControl, "STOP", zmq::send_flags::sndmore);
+  m_zmqSvc->send(*m_allenControl, m_allenConfig->stopTimeout.value());
 
-  m_zmqSvc->send(*m_allenControl, "STOP");
+  // This will stop input
+  auto sc = OnlineApplication::stop();
+  if (!sc) return sc;
 
   zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
   m_zmqSvc->poll(&items[0], 1, -1);
@@ -94,12 +104,11 @@ int AllenApplication::stop()
       return Online::ONLINE_ERROR;
     }
   }
-
-  return OnlineApplication::stop();
+  return sc;
 }
 
 /// Cancel the application: Cancel IO request/Event loop
-int AllenApplication::cancel() { return 1; }
+int AllenApplication::cancel() { return Online::ONLINE_OK; }
 
 /// Internal: Initialize the application            (NOT_READY  -> READY)
 int AllenApplication::configureApplication()
@@ -161,24 +170,50 @@ int AllenApplication::configureApplication()
     return Online::ONLINE_ERROR;
   }
   m_updater = dynamic_cast<Allen::NonEventData::IUpdater*>(updater.get());
-  if (updater == nullptr) {
+  if (m_updater == nullptr) {
     m_logger->error("Failed to cast AllenUpdater");
     return Online::ONLINE_ERROR;
   }
 
-  if (m_allenConfig->source.value() == "MPI") {
-    auto success = initMPI();
-    if (!success) {
-      m_logger->error("Failed to initialize MPI");
+  SmartIF<IService> mepProvider = sloc->service<IService>("MEPProvider");
+  if (!mepProvider.get()) {
+    m_logger->error("Failed to retrieve MEPProvider.");
+    return Online::ONLINE_ERROR;
+  }
+
+  std::string value;
+  auto sc = mepProvider.as<IProperty>()->getProperty("NSlices", value);
+  if (!sc.isSuccess()) {
+    m_logger->error("Failed to get NSlices property from MEPProvider");
+    return Online::ONLINE_ERROR;
+  }
+  else {
+    using Gaudi::Parsers::parse;
+    sc = parse(m_nSlices, value);
+    if (!sc.isSuccess()) {
+      m_logger->error("Failed to parse NSlices property");
       return Online::ONLINE_ERROR;
     }
   }
 
-  m_provider = makeProvider();
-  if (!m_provider) {
+  m_provider = dynamic_cast<IInputProvider*>(mepProvider.get());
+  if (m_provider == nullptr) {
+    m_logger->error("Failed to cast MEPProvider");
     return Online::ONLINE_ERROR;
   }
 
+  m_json = resolveEnvVars(m_allenConfig->json);
+  fs::path json{m_json};
+  if (!fs::exists(json) || !fs::is_regular_file(json)) {
+    m_logger->error("Allen JSON configuration file does not exist or is not a regular file");
+    return Online::ONLINE_ERROR;
+  }
+
+  m_output = makeOutput();
+  if (!m_allenConfig->output.value().empty() && !m_output) {
+    m_logger->error("Failed to create output for "s + m_allenConfig->output.value());
+  }
+
   m_allenControl = m_zmqSvc->socket(zmq::PAIR);
   m_allenControl->bind(m_controlConnection.c_str());
 
@@ -268,123 +303,69 @@ int AllenApplication::continueProcessing()
   return OnlineApplication::continueProcessing();
 }
 
-bool AllenApplication::initMPI()
-{
-#ifdef HAVE_MPI
-  // MPI initialization
-  auto len = name().length();
-  int provided = 0;
-  m_mpiArgv = new char*[1];
-  m_mpiArgv[0] = new char[len];
-  ::strncpy(m_mpiArgv[0], name().c_str(), len);
-  MPI_Init_thread(&m_mpiArgc, &m_mpiArgv, MPI_THREAD_MULTIPLE, &provided);
-  if (provided != MPI_THREAD_MULTIPLE) {
-    m_logger->error("Failed to initialize MPI multi thread support.");
-    return false;
-  }
-
-  // Communication size
-  int comm_size = 0;
-  MPI_Comm_size(MPI_COMM_WORLD, &comm_size);
-  if (comm_size > MPI::comm_size) {
-    std::string e = "This program requires at most "s + std::to_string(MPI::comm_size) + " processes.";
-    m_logger->error(e.c_str());
-    return false;
-  }
-
-  // MPI: Who am I?
-  MPI_Comm_rank(MPI_COMM_WORLD, &m_rank);
-
-  if (m_rank != MPI::receiver) {
-    m_logger->error("AllenApplication can only function as MPI receiver.");
-    return false;
-  }
-  else {
-    return true;
-  }
-#else
-  m_logger->error("MPI requested, but Allen was not built with MPI support.");
-  return false;
-#endif
-}
-
 void AllenApplication::allenLoop()
 {
-
-  auto json = resolveEnvVars(m_allenConfig->json);
   auto paramDir = resolveEnvVars(m_allenConfig->paramDir);
 
   //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
   //--device 23:00.0
-  std::map<std::string, std::string> allen_options = {{"events-per-slice", std::to_string(m_allenConfig->eps.value())},
-                                                      {"c", std::to_string(m_allenConfig->check.value())},
-                                                      {"v", std::to_string(6 - m_config->outputLevel())},
+  std::map<std::string, std::string> allen_options = {{"v", std::to_string(6 - m_config->outputLevel())},
                                                       {"t", std::to_string(m_allenConfig->nThreads.value())},
                                                       {"geometry", paramDir},
-                                                      {"configuration", json},
-                                                      {"device", m_allenConfig->device.value()}};
+                                                      {"configuration", m_json},
+                                                      {"device", m_allenConfig->device.value()},
+                                                      {"s", std::to_string(m_nSlices)}};
+
+  allen(allen_options, m_updater, m_provider, m_output.get(), m_zmqSvc.get(), m_controlConnection);
+}
 
-  if (!m_allenConfig->output.value().empty()) {
-    allen_options["output-file"] = m_allenConfig->output.value();
+std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
+{
+  size_t n_lines = m_allenConfig->nLines.value();
+  if (n_lines == 0) {
+    m_logger->warning("No lines configured in Allen sequence");
   }
 
-  if (m_allenConfig->nSlices.value() != 0) {
-    allen_options["s"] = std::to_string(m_allenConfig->nSlices.value());
+  auto output = m_allenConfig->output.value();
+  std::string connection;
+  auto p = output.find("://");
+  std::string output_type;
+  if (p == std::string::npos) {
+    output_type = "file";
+    connection = output;
+  }
+  else {
+    output_type = output.substr(0, p);
+    connection = output.substr(p + 3, std::string::npos);
   }
 
-  if (m_allenConfig->source.value() == "Files" && m_allenConfig->input.value().empty()) {
-    m_logger->throwError("No input files specified");
+  if (output_type == "file" || output_type == "tcp") {
+    std::map<std::string, std::string> options = {{"configuration", m_json},
+                                                  {"output-file", output}};
+    return Allen::output_handler(m_provider, m_zmqSvc.get(), std::move(options));
   }
+  else if (output_type == "mbm") {
+    if (m_allenConfig->partitionBuffers.value()) {
+      connection += "_";
+      auto const partition = m_allenConfig->partition.value();
+      std::stringstream hexID;
+      hexID << std::hex << m_allenConfig->partitionID.value();
+      connection += partition.empty() ? hexID.str() : m_allenConfig->partition.value();
+    }
 
-  allen(allen_options, m_updater, m_provider.get(), m_zmqSvc.get(), m_controlConnection);
-}
+    auto output_logger = m_logger->clone("MBMOutput", m_config->outputLevel());
 
-std::unique_ptr<IInputProvider> AllenApplication::makeProvider()
-{
-  std::unordered_map<std::string, MEPProviderSource> sources{{"MBM", MEPProviderSource::MBM},
-                                                             {"MPI", MEPProviderSource::MPI},
-                                                             {"Files", MEPProviderSource::Files}};
-
-  // FIXME: Do this in AllenConfiguration in the proper way with
-  // property parsing
-  if (!sources.count(m_allenConfig->source.value())) {
-    m_logger->error(std::string{"MEPProviderSource must be one of MBM, MPI, Files. Got "} + m_allenConfig->source.value());
-    return {};
+    return std::make_unique<MBMOutput>(std::move(output_logger),
+                                       m_provider,
+                                       connection,
+                                       RTL::processName(),
+                                       m_allenConfig->partitionID.value(),
+                                       n_lines,
+                                       m_allenConfig->checksum.value());
   }
-
-  MEPProviderConfig config {false,                // verify MEP checksums
-                            10,                   // number of read buffers
-                            m_allenConfig->transpose.value() ? 4u : 1u, // number of transpose threads
-                            4,                                          // MPI sliding window size
-                            sources[m_allenConfig->source.value()],     // Receive from MPI, MBM or files
-                            m_allenConfig->nonStop.value(),             // Loop over input non-stop
-                            m_allenConfig->transpose.value(),           // MEPs should be transposed to Allen layout
-                            m_allenConfig->runChanges.value(),          // Whether to split slices by run number
-                            m_allenConfig->receivers.value(),           // Map of receiver to MPI rank to receive from
-                            m_allenConfig->bufferName.value(),          // MBM buffer name
-                            m_allenConfig->partition.value(),           // Partition name
-                            m_allenConfig->partitionID.value(),         // Parition ID
-                            m_allenConfig->partitionBuffers.value(),    // Partition buffers using partition name
-                            m_allenConfig->requests.value()};           // MBM requests
-
-  std::unordered_set<BankTypes> bankTypes;
-  for (auto bt : m_allenConfig->bankTypes.value()) {
-    auto it = Allen::bank_types.find(bt);
-    if (it == Allen::bank_types.end()) {
-      m_logger->error(std::string{"No Allen bank type defined for requested bank type "} + LHCb::RawBank::typeName(bt));
-      return {};
-    }
-    else {
-      bankTypes.insert(it->second);
-    }
+  else {
+    m_logger->error("Unknown output type: "s + output_type);
+    return {};
   }
 
-  auto io_logger = m_logger->clone("MEPProvider", m_config->outputLevel());
-  return std::make_unique<MEPProvider>(std::move(io_logger),
-                                       name(),
-                                       m_allenConfig->nSlices.value(),
-                                       m_allenConfig->eps.value(),
-                                       m_allenConfig->input.value(),
-                                       bankTypes,
-                                       config);
 }
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index d294298eb..f59b80557 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -21,6 +21,8 @@
 #include <GaudiOnline/OnlineApplication.h>
 #include <ZeroMQ/IZeroMQSvc.h>
 
+#include <Allen/OutputHandler.h>
+
 #include "AllenConfiguration.h"
 
 class AllenApplication : public Online::OnlineApplication {
@@ -64,7 +66,7 @@ public:
 
 private:
 
-  std::unique_ptr<IInputProvider> makeProvider();
+  std::unique_ptr<OutputHandler> makeOutput();
 
   /// Reference to the monitoring service
   SmartIF<IMonitorSvc> m_monSvc;
@@ -82,7 +84,10 @@ private:
 
   std::string m_controlConnection = "inproc://AllenApplicationControl";
 
-  std::unique_ptr<IInputProvider> m_provider;
+  IInputProvider* m_provider = nullptr;
+  std::unique_ptr<OutputHandler> m_output;
+
+  size_t m_nSlices = 0;
 
   // dlopen stuff to workaround segfault in genconf.exe
   // void* m_handle = nullptr;
@@ -93,9 +98,5 @@ private:
   std::thread m_allenThread;
   std::optional<zmq::socket_t> m_allenControl;
 
-#ifdef HAVE_MPI
-  char** m_mpiArgv = nullptr;
-  int m_mpiArgc = 1;
-  int m_rank = -1;
-#endif
+  std::string m_json;
 };
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index ee2150dc0..078dae923 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -1,7 +1,6 @@
 #pragma once
 
 #include <GaudiKernel/Service.h>
-#include <Dumpers/RawBankProperty.h>
 
 class AllenConfiguration : public Service {
 public:
@@ -19,32 +18,18 @@ public:
 
   ~AllenConfiguration();
 
-  Gaudi::Property<int> eps {this, "EventsPerSlice", 1000};
-  Gaudi::Property<bool> nonStop {this, "NonStop", true};
-  Gaudi::Property<bool> transpose {this, "TransposeMEP", false};
-  Gaudi::Property<std::string> source {this, "Source", "MBM"};
-  Gaudi::Property<std::map<std::string, int>> receivers {this, "Receivers"};
-  Gaudi::Property<bool> check {this, "CheckMC", false};
+  Gaudi::Property<float> stopTimeout {this, "StopTimeout", 5.};
   Gaudi::Property<unsigned int> nThreads {this, "NThreads", 8};
-  Gaudi::Property<unsigned int> nSlices {this, "NSlices", 16};
   Gaudi::Property<std::string> output {this, "Output", ""};
   Gaudi::Property<bool> runChanges {this, "EnableRunChanges", true};
   Gaudi::Property<std::string> device {this, "Device", "0"};
   Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json"};
+  Gaudi::Property<size_t> nLines{this, "NLines", 0ul};
   Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
   Gaudi::Property<std::vector<std::string>> input {this, "Input"};
-  Gaudi::Property<std::string> bufferName {this, "BufferName", "Events"};
-  Gaudi::Property<std::string> partition {this, "Partition"};
-  Gaudi::Property<unsigned> partitionID {this, "PartitionID"};
-  Gaudi::Property<bool> partitionBuffers {this, "PartitionBuffers", false};
-  Gaudi::Property<std::vector<std::string>> requests {this, "Requests"};
 
-  Gaudi::Property<std::unordered_set<LHCb::RawBank::BankType>> bankTypes{this, "BankTypes",
-                                                                  {LHCb::RawBank::VP,
-                                                                   LHCb::RawBank::UT,
-                                                                   LHCb::RawBank::FTCluster,
-                                                                   LHCb::RawBank::Muon,
-                                                                   LHCb::RawBank::ODIN,
-                                                                   LHCb::RawBank::EcalPacked,
-                                                                   LHCb::RawBank::HcalPacked}};
+  Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
+  Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
+  Gaudi::Property<bool> partitionBuffers {this, "PartitionBuffers", true};
+  Gaudi::Property<std::string> partition {this, "Partition", ""};
 };
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
new file mode 100644
index 000000000..dd7af51f1
--- /dev/null
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -0,0 +1,106 @@
+#include <AllenOnline/MBMOutput.h>
+#include <MDF/MDFHeader.h>
+#include <MDF/RawEventHelpers.h>
+
+MBMOutput::MBMOutput(
+    std::unique_ptr<RTL::Logger>&& logger,
+    IInputProvider const* inputProvider,
+    std::string bufferName,
+    std::string processName,
+    unsigned partitionID,
+    size_t const nLines,
+    bool const checksum) :
+    OutputHandler {inputProvider, bufferName, nLines},
+    m_logger{std::move(logger)},
+    m_bufferName {std::move(bufferName)},
+    m_processName{std::move(processName)},
+    m_partitionID{partitionID},
+    m_checksum{checksum}
+{
+}
+
+int MBMOutput::spaceCallback(void* /* param */)
+{
+  return MBM_NORMAL;
+}
+
+std::tuple<size_t, gsl::span<char>> MBMOutput::buffer(size_t buffer_size)
+{
+  if (m_cancelled) {
+    return {0, {}};
+  }
+
+  int* buf = nullptr;
+  auto sc = ::mbm_get_space_a(m_bmID, buffer_size, &buf, spaceCallback, this);
+  if ( sc == MBM_NORMAL )  {
+    sc = ::mbm_wait_space(m_bmID);
+    if ( sc == MBM_NORMAL )   {
+      m_buffer = {reinterpret_cast<char*>(buf), static_cast<events_size>(buffer_size)};
+      return {0, m_buffer};
+    }
+  }
+  return {0, {}};
+}
+
+bool MBMOutput::start()
+{
+  m_cancelled = false;
+  m_bmID = ::mbm_include_write(m_bufferName.c_str(), m_processName.c_str(), m_partitionID, BM_COM_FIFO);
+  if (m_bmID == MBM_INV_DESC)   {
+    return m_logger->error("MBMOutput: failed to connect to MBM buffer %s!",
+                           m_bufferName.c_str());
+  }
+  return true;
+}
+
+bool MBMOutput::write_buffer(size_t)
+{
+  if (m_cancelled) {
+    return false;
+  }
+
+  if (m_checksum) {
+    auto* header = reinterpret_cast<LHCb::MDFHeader*>(&m_buffer[0]);
+    auto const skip = 4 * sizeof(int);
+    auto c = LHCb::hash32Checksum(m_buffer.data() + skip, m_buffer.size() - skip);
+    header->setChecksum(c);
+  }
+
+  unsigned int mask[] = { ~0x0U, ~0x0U, ~0x0U, m_partitionID };
+  void* free_address = nullptr;
+  long free_len = 0;
+
+  auto sc = ::mbm_declare_event(m_bmID, m_buffer.size_bytes(),
+                                EVENT_TYPE_BURST,
+                                mask, 0, &free_address, &free_len, m_partitionID);
+  if (sc == MBM_REQ_CANCEL) {
+    return false;
+  }
+  else if ( sc == MBM_NORMAL )  {
+    sc = ::mbm_send_space(m_bmID);
+    if ( sc == MBM_REQ_CANCEL ) {
+      return false;
+    }
+    else {
+      return true;
+    }
+  } else {
+    return m_logger->error("MBMOutput: failed to write buffer %d!", sc);
+  }
+}
+
+void MBMOutput::cancel() {
+  if (m_bmID != MBM_INV_DESC) {
+    ::mbm_cancel_request(m_bmID);
+  }
+  m_cancelled = true;
+}
+
+bool MBMOutput::stop()
+{
+  m_cancelled = false;
+  if (m_bmID != MBM_INV_DESC) {
+    ::mbm_exclude(m_bmID);
+  }
+  return true;
+}
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 0fa3c8923..2ad214e8f 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -5,6 +5,7 @@
 #include <Allen/Logger.h>
 #include <Allen/InputProvider.h>
 #include <Allen/BankTypes.h>
+#include <Allen/BankMapping.h>
 #include <Allen/Timer.h>
 #include <Allen/SliceUtils.h>
 #include <Backend/BackendCommon.h>
@@ -27,6 +28,8 @@
 #include <hwloc.h>
 #endif
 
+DECLARE_COMPONENT(MEPProvider)
+
 // The MEPProvider has three possible sources: MPI, the BufferManager
 // and files.
 //
@@ -45,71 +48,42 @@
 // allocated by the BufferManager is registered with the device
 // runtime.
 
-MEPProvider::MEPProvider(
-  std::unique_ptr<RTL::Logger>&& logger,
-  std::string proc_name,
-  size_t n_slices,
-  size_t events_per_slice,
-  std::vector<std::string> connections,
-  std::unordered_set<BankTypes> const& bank_types,
-  MEPProviderConfig config) noexcept(false) :
-InputProvider {n_slices, events_per_slice, bank_types,
-  config.transpose_mep ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, {}},
-  m_logger(std::move(logger)), m_process_name{std::move(proc_name)}, m_buffer_status(config.n_buffers), m_slice_free(n_slices, true),
-  m_banks_count {0}, m_event_ids {n_slices}, m_connections {std::move(connections)}, m_config {config}
+MEPProvider::MEPProvider(std::string name, ISvcLocator* loc)
+  : Service {name, loc},
+    m_banks_count {0}
 {
 
-  if (m_config.transpose_mep) {
-    m_logger->info("Providing events in Allen layout by transposing MEPs");
-  }
-  else {
-    m_logger->info("Providing events in MEP layout");
-  }
-
-  m_buffer_transpose = m_buffer_status.begin();
-  m_buffer_reading = m_buffer_status.begin();
-
-  if (m_config.source == MEPProviderSource::MPI) {
-    init_mpi();
-  }
-  else {
-    m_read_buffers.resize(m_config.n_buffers);
-    m_net_slices.resize(m_config.n_buffers);
-  }
-
-  // Initialize the current input filename
-  m_current = m_connections.begin();
-
-  // Allocate space to store event ids
-  for (size_t n = 0; n < n_slices; ++n) {
-    m_event_ids[n].reserve(events_per_slice);
-  }
-
-  // Cached bank LHCb bank type to Allen bank type mapping
-  m_bank_ids = Allen::bank_ids();
+  m_bufferConfig.declareUpdateHandler([this](auto &) -> void {
+    // Sanity check on the number of buffers and threads
+    auto [n_buf, n_transpose] = m_bufferConfig.value();
+    if (n_buf < 2) {
+      warning() << "Too few read buffers requested, setting it to 2" << endmsg;
+      n_buf = 2;
+    }
 
-  // Reserve 1MB for decompression
-  m_compress_buffer.reserve(1u * MB);
+    if (n_transpose > n_buf - 1) {
+      warning() << "Too many transpose threads requested with respect "
+                << "to the number of read buffers; reducing the number of threads to "
+                << n_buf - 1 << endmsg;
+      n_transpose = n_buf - 1;
+    }
+    m_bufferConfig.set({n_buf, n_transpose});
+  });
 
-  // Sanity check on the number of buffers and threads
-  if (m_config.n_buffers <= 1) {
-    m_logger->warning("too few read buffers requested, setting it to 2");
-    m_config.n_buffers = 2;
-  }
+  m_lhcb_bank_types.declareUpdateHandler([this](auto &) -> void {
+    for (auto bt : m_lhcb_bank_types) {
+      auto it = Allen::bank_types.find(bt);
+      if (it == Allen::bank_types.end()) {
+        throw GaudiException(this->name(), "No Allen bank type defined for requested bank type "s
+                             + LHCb::RawBank::typeName(bt), StatusCode::FAILURE);
+      }
+      else {
+        m_bank_types.insert(it->second);
+      }
+    }
+  });
 
-  if (m_config.n_transpose_threads > m_config.n_buffers - 1) {
-    m_logger->warning(std::string{"too many transpose threads requested with respect "} +
-                      "to the number of read buffers; reducing the number of threads to "
-                      + std::to_string(m_config.n_buffers - 1));
-    m_config.n_transpose_threads = m_config.n_buffers - 1;
-  }
 
-  // Start the transpose threads
-  if (m_transpose_threads.empty() && !m_read_error) {
-    for (size_t i = 0; i < m_config.n_transpose_threads; ++i) {
-      m_transpose_threads.emplace_back([this, i] { transpose(i); });
-    }
-  }
 }
 
 EventIDs MEPProvider::event_ids(size_t slice_index, std::optional<size_t> first, std::optional<size_t> last) const
@@ -129,7 +103,7 @@ BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) cons
   for (auto s : banks) {
     spans.emplace_back(s);
   }
-  std::get<1>(bno) = m_config.transpose_mep ? offsets[offsets_size - 1] : data_size;
+  std::get<1>(bno) = m_transpose_mep.value() ? offsets[offsets_size - 1] : data_size;
   std::get<2>(bno) = offsets;
   std::get<3>(bno) = m_banks_version[ib];
   return bno;
@@ -173,15 +147,13 @@ MEPProvider::get_slice(std::optional<unsigned int> timeout)
   auto n_writable = count_writable();
   done = ((m_transpose_done && m_transposed.empty()) || m_stopping) && n_writable == m_buffer_status.size();
 
-  if (timed_out && logger::verbosity() >= logger::verbose) {
-    this->debug_output(
-      "get_slice timed out; error " + std::to_string(m_read_error) + " done " + std::to_string(done) + " n_filled " +
-      std::to_string(n_filled));
+  if (timed_out && msgLevel(MSG::DEBUG)) {
+    debug() << "get_slice timed out; error " << m_read_error << " done " << done
+            << " n_filled " << n_filled << endmsg;
   }
-  else if (!timed_out) {
-    this->debug_output(
-      "get_slice returning " + std::to_string(slice_index) + "; error " + std::to_string(m_read_error) + " done " +
-      std::to_string(done) + " n_filled " + std::to_string(n_filled));
+  else if (!timed_out && msgLevel(MSG::DEBUG)) {
+    debug() << "get_slice returning " << slice_index << "; error " << m_read_error << " done "
+            << done << " n_filled " << n_filled << endmsg;
   }
 
   return {!m_read_error, done, timed_out, slice_index, m_read_error ? 0 : n_filled, run_no};
@@ -213,18 +185,18 @@ void MEPProvider::slice_free(size_t slice_index)
          }) == m_slice_to_buffer.end())) {
         status.writable = true;
         set_writable = true;
-        if (m_config.source == MEPProviderSource::MBM) {
+        if (m_source == MEP::ProviderSource::MBM) {
           ::mbm_free_event(m_bmIDs[i_buffer]);
         }
       }
     }
   }
   if (freed) {
-    this->debug_output("Freed slice " + std::to_string(slice_index));
+    if (msgLevel(MSG::DEBUG)) debug() << "Freed slice " << slice_index << endmsg;
     m_slice_cond.notify_one();
   }
   if (set_writable) {
-    this->debug_output("Set buffer " + std::to_string(i_buffer) + " writable");
+    if (msgLevel(MSG::DEBUG)) debug() << "Set buffer " << i_buffer << " writable" << endmsg;
     m_mpi_cond.notify_one();
   }
 }
@@ -285,43 +257,94 @@ void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event,
   }
 }
 
-int MEPProvider::start()
+StatusCode MEPProvider::initialize()
 {
-  if (m_started) {
-    return true;
+  m_slice_free.resize(m_nslices.value(), true);
+  m_event_ids.resize(m_nslices.value());
+
+  m_buffer_status.resize(n_buffers());
+
+  init_input(m_nslices, m_events_per_slice, m_bank_types,
+             m_transpose_mep.value() ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, {});
+
+  if (m_transpose_mep) {
+    info() << "Providing events in Allen layout by transposing MEPs" << endmsg;
   }
   else {
-    // start MPI receive, MEP reading thread or BM thread
-    if (m_config.source == MEPProviderSource::MPI) {
-      #ifdef HAVE_MPI
-      m_input_threads.emplace_back(&MEPProvider::mpi_read, this);
-      #else
-      throw StrException {"MPI requested, but no MPI support built in."};
-      #endif
+    info() << "Providing events in MEP layout" << endmsg;;
+  }
+
+  m_buffer_transpose = m_buffer_status.begin();
+  m_buffer_reading = m_buffer_status.begin();
+
+  StatusCode sc = StatusCode::SUCCESS;
+  if (m_source == MEP::ProviderSource::MPI) {
+    sc = init_mpi();
+    if (!sc.isSuccess()) return sc;
+  }
+  else {
+    m_read_buffers.resize(n_buffers());
+    m_net_slices.resize(n_buffers());
+  }
+
+  // Initialize the current input filename
+  m_current = m_connections.begin();
+
+  // Allocate space to store event ids
+  for (size_t n = 0; n < m_nslices.value(); ++n) {
+    m_event_ids[n].reserve(m_events_per_slice);
+  }
+
+  // Cached bank LHCb bank type to Allen bank type mapping
+  m_bank_ids = Allen::bank_ids();
+
+  return sc;
+}
+
+StatusCode MEPProvider::start()
+{
+  if (m_started) {
+    return StatusCode::SUCCESS;
+  }
+
+  // Start the transpose threads
+  if (m_transpose_threads.empty() && !m_read_error) {
+    for (int i = 0; i < std::get<1>(m_bufferConfig.value()); ++i) {
+      m_transpose_threads.emplace_back([this, i] { transpose(i); });
     }
-    else if (m_config.source == MEPProviderSource::Files) {
-      m_input_threads.emplace_back(&MEPProvider::mep_read, this);
-    } else {
-      auto sc = init_bm();
-      if (sc != MBM_NORMAL) {
-        return false;
-      }
-      for (size_t b = 0; b < m_config.n_buffers; ++b) {
-        m_input_threads.emplace_back(&MEPProvider::bm_read, this);
-      }
+  }
+
+  // start MPI receive, MEP reading thread or BM thread
+  if (m_source == MEP::ProviderSource::MPI && m_input_threads.empty()) {
+#ifdef HAVE_MPI
+    m_input_threads.emplace_back(&MEPProvider::mpi_read, this);
+#else
+    throw StrException {"MPI requested, but no MPI support built in."};
+#endif
+  }
+  else if (m_source == MEP::ProviderSource::Files  && m_input_threads.empty()) {
+    m_input_threads.emplace_back(&MEPProvider::mep_read, this);
+  }
+  else if (m_source == MEP::ProviderSource::MBM) {
+    auto sc = init_bm();
+    if (sc != MBM_NORMAL) {
+      return StatusCode::FAILURE;
+    }
+    for (size_t b = 0; b < n_buffers(); ++b) {
+      m_input_threads.emplace_back(&MEPProvider::bm_read, this);
     }
+  }
 
-    std::unique_lock<std::mutex> lock {m_control_mutex};
-    this->debug_output("Starting", 0);
+  std::unique_lock<std::mutex> lock {m_control_mutex};
+  debug() << "Starting" << endmsg;
 
-    m_started = true;
-    m_stopping = false;
-    m_control_cond.notify_one();
-    return true;
-  }
+  m_started = true;
+  m_stopping = false;
+  m_control_cond.notify_one();
+  return StatusCode::SUCCESS;
 };
 
-int MEPProvider::stop()
+StatusCode MEPProvider::stop()
 {
   {
     std::unique_lock<std::mutex> lock {m_control_mutex};
@@ -329,26 +352,65 @@ int MEPProvider::stop()
     m_started = false;
   }
 
-  if (m_config.source == MEPProviderSource::MBM) {
+  if (m_source == MEP::ProviderSource::MBM) {
     // Cancel all requests to the buffer manager for those who are waiting
     std::unique_lock<std::mutex> lock {m_mpi_mutex};
     for (size_t b = 0; b < m_buffer_status.size(); ++b) {
       if (!m_buffer_status[b].writable) ::mbm_cancel_request(m_bmIDs[b]);
     }
+
+    for (auto& thread : m_input_threads) {
+      thread.join();
+    }
+    m_input_threads.clear();
+
+    for (auto bmid : m_bmIDs) {
+      if (bmid != MBM_INV_DESC) {
+        ::mbm_exclude(bmid);
+      }
+    }
   }
 
   // Make sure all threads wait for start in case they were waiting
   // for a buffer
   m_mpi_cond.notify_all();
 
-  return true;
+  return StatusCode::SUCCESS;
 };
 
-void MEPProvider::init_mpi()
+StatusCode MEPProvider::init_mpi()
 {
   #ifdef HAVE_MPI
-  auto const& receivers = m_config.receivers;
-  m_domains.reserve(receivers.size());
+
+  // MPI initialization
+  auto len = name().length();
+  int provided = 0;
+  m_mpiArgv = new char*[1];
+  m_mpiArgv[0] = new char[len];
+  ::strncpy(m_mpiArgv[0], name().c_str(), len);
+  MPI_Init_thread(&m_mpiArgc, &m_mpiArgv, MPI_THREAD_MULTIPLE, &provided);
+  if (provided != MPI_THREAD_MULTIPLE) {
+    error() << "Failed to initialize MPI multi thread support." << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  // Communication size
+  int comm_size = 0;
+  MPI_Comm_size(MPI_COMM_WORLD, &comm_size);
+  if (comm_size > MPI::comm_size) {
+    error() << "This program requires at most " << MPI::comm_size << " processes." << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  // MPI: Who am I?
+  MPI_Comm_rank(MPI_COMM_WORLD, &m_rank);
+
+  if (m_rank != MPI::receiver) {
+    error() << "AllenApplication can only function as MPI receiver." << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  m_domains.reserve(m_receivers.size());
 
   // Allocate and initialize topology object.
   hwloc_topology_init(&m_topology);
@@ -365,40 +427,42 @@ void MEPProvider::init_mpi()
 
   hwloc_obj_t osdev = nullptr;
 
-  if (!receivers.empty()) {
+  if (!m_receivers.empty()) {
     // Find NUMA domain of receivers
     while ((osdev = hwloc_get_next_osdev(m_topology, osdev))) {
       // We're interested in InfiniBand cards
       if (osdev->attr->osdev.type == HWLOC_OBJ_OSDEV_OPENFABRICS) {
         auto parent = hwloc_get_non_io_ancestor_obj(m_topology, osdev);
-        auto it = receivers.find(osdev->name);
-        if (it != receivers.end()) {
+        auto it = m_receivers.find(osdev->name);
+        if (it != m_receivers.end()) {
           m_domains.emplace_back(it->second, parent->os_index);
-          this->debug_output(
-            "Located receiver device "s + it->first + " in NUMA domain " + std::to_string(parent->os_index));
+          debug() << "Located receiver device " << it->first
+                  << " in NUMA domain " << parent->os_index << endmsg;
         }
       }
     }
-    if (m_domains.size() != receivers.size()) {
-      throw StrException {"Failed to locate some receiver devices "};
+    if (m_domains.size() != m_receivers.size()) {
+      error() << "Failed to locate some receiver devices " << endmsg;
+      return StatusCode::FAILURE;
     }
   }
   else {
-    throw StrException {"MPI requested, but no receivers specified"};
+    error() << "MPI requested, but no receivers specified" << endmsg;
+    return StatusCode::FAILURE;
   }
 
   // Get last node. There's always at least one.
   [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
   assert(static_cast<size_t>(n_numa) == m_domains.size());
 
-  std::vector<hwloc_obj_t> numa_objs(m_config.n_receivers());
-  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
+  std::vector<hwloc_obj_t> numa_objs(m_receivers.size());
+  for (size_t receiver = 0; receiver < m_receivers.size(); ++receiver) {
     int numa_node = std::get<1>(m_domains[receiver]);
     numa_objs[receiver] = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
   }
 
-  std::vector<size_t> packing_factors(m_config.n_receivers());
-  for (size_t receiver = 0; receiver < m_config.n_receivers(); ++receiver) {
+  std::vector<size_t> packing_factors(m_receivers.size());
+  for (size_t receiver = 0; receiver < m_receivers.size(); ++receiver) {
     auto const receiver_rank = std::get<0>(m_domains[receiver]);
     MPI_Recv(
       &packing_factors[receiver],
@@ -413,7 +477,9 @@ void MEPProvider::init_mpi()
   if (!std::all_of(packing_factors.begin(), packing_factors.end(), [v = packing_factors.back()](auto const p) {
         return p == v;
       })) {
-    throw StrException {"All packing factors must be the same"};
+    error() << "All MEPs must have the same packing factor" << endmsg;
+    return StatusCode::FAILURE;
+
   }
   else {
     m_packing_factor = packing_factors.back();
@@ -422,20 +488,21 @@ void MEPProvider::init_mpi()
   // Allocate as many net slices as configured, of expected size
   // Packing factor can be done dynamically if needed
   size_t n_bytes = std::lround(m_packing_factor * average_event_size * bank_size_fudge_factor * kB);
-  for (size_t i = 0; i < m_config.n_buffers; ++i) {
+  for (size_t i = 0; i < n_buffers(); ++i) {
     char* contents = nullptr;
     MPI_Alloc_mem(n_bytes, MPI_INFO_NULL, &contents);
 
     // Only bind explicitly if there are multiple receivers,
     // otherwise assume a memory allocation policy is in effect
     if (m_domains.size() > 1) {
-      auto numa_node = i % m_config.n_receivers();
+      auto numa_node = i % m_receivers.size();
       auto const& numa_obj = numa_objs[numa_node];
       auto s = hwloc_set_area_membind(
         m_topology, contents, n_bytes, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
       if (s != 0) {
-        throw StrException {"Failed to bind memory to node "s + std::to_string(numa_obj->os_index) + " " +
-                            strerror(errno)};
+        error() << "Failed to bind memory to node " << numa_obj->os_index << " "
+                << strerror(errno) << endmsg;
+        return StatusCode::FAILURE;
       }
     }
 
@@ -449,34 +516,34 @@ void MEPProvider::init_mpi()
                  n_bytes});
     m_mpi_buffers.emplace_back(contents);
   }
+  return StatusCode::SUCCESS;
   #else
-  m_logger->error("MPI requested, but Allen was not built with MPI support.");
-  throw std::runtime_error {"No MPI supoprt"};
+  error() << "MPI requested, but Allen was not built with MPI support." << endmsg;
+  return StatusCode::FAILURE;
   #endif
 }
 
 int MEPProvider::init_bm()
 {
-  auto buffer_name = m_config.bufferName;
-  if ( m_config.partitionBuffers ) {
+  auto buffer_name = m_bufferName.value();
+  if ( m_partitionBuffers.value() ) {
     std::stringstream stream;
-    stream << std::hex << m_config.partitionID;
+    stream << std::hex << m_partitionID.value();
     buffer_name += "_";
-    buffer_name += m_config.partition.empty() ? stream.str() : m_config.partition;
+    buffer_name += m_partition.value().empty() ? stream.str() : m_partition.value();
   }
 
-
-  for (size_t b = 0; b < m_config.n_buffers; ++b) {
+  for (size_t b = 0; b < n_buffers(); ++b) {
     auto pn = m_process_name + "." + std::to_string(b);
     BMID bmid;
     if (b == 0) {
-      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), m_config.partitionID, BM_COM_FIFO);
+      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), m_partitionID, BM_COM_FIFO);
     } else {
-      bmid = ::mbm_connect(m_bmIDs[0], pn.c_str(), m_config.partitionID);
+      bmid = ::mbm_connect(m_bmIDs[0], pn.c_str(), m_partitionID);
     }
 
     if (bmid == MBM_INV_DESC)   {
-      return m_logger->error("MBM: Failed to connect to MBM buffer %s!", buffer_name.c_str());
+      error() << "MBM: Failed to connect to MBM buffer " << buffer_name << endmsg;
       return MBM_ERROR;
     } else if (b == 0) {
       // register buffer manager memory with the device runtime
@@ -486,16 +553,16 @@ int MEPProvider::init_bm()
       if (sc == MBM_NORMAL) {
         Allen::host_register(const_cast<char*>(mem), mem_size, Allen::hostRegisterDefault);
       } else {
-        m_logger->error("MBM: Failed to obtain buffer start and size, MBM");
+        error() << "MBM: Failed to obtain buffer start and size, MBM" << endmsg;
         return MBM_ERROR;
       }
     }
-    for(auto r : m_config.requests) {
+    for(auto r : m_requests) {
       MBM::Requirement rq{r};
       int sc = ::mbm_add_req(bmid, rq.evtype, rq.trmask, rq.vetomask, rq.maskType,
                              rq.userType, rq.freqType, rq.freq);
       if (sc != MBM_NORMAL)   {
-        m_logger->error("MBM: Failed to add MBM requirement: %s",r.c_str());
+        error() << "MBM: Failed to add MBM requirement: " << r << endmsg;
         return MBM_ERROR;
       }
     }
@@ -559,7 +626,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
       throw std::out_of_range {std::string {"Bank type "} + std::to_string(ib) + " has no known size"};
     }
     // In case of direct MEP output, no memory should be allocated.
-    if (!m_config.transpose_mep) {
+    if (!m_transpose_mep.value()) {
       auto it = std::find(m_bank_ids.begin(), m_bank_ids.end(), to_integral(bank_type));
       auto lhcb_type = std::distance(m_bank_ids.begin(), it);
       auto n_blocks = m_banks_count[lhcb_type];
@@ -571,12 +638,12 @@ bool MEPProvider::allocate_storage(size_t i_read)
       return {std::lround(it->second * aps * bank_size_fudge_factor * kB), eps};
     }
   };
-  m_slices = allocate_slices(this->n_slices(), types(), size_fun);
+  m_slices = allocate_slices(m_nslices, types(), size_fun);
 
-  m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(this->n_slices(), std::make_tuple(-1, 0ul, 0ul));
+  m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(m_nslices, std::tuple{-1, 0ul, 0ul});
 
   if (!count_success) {
-    m_logger->error("Failed to determine bank counts");
+    error() << "Failed to determine bank counts" << endmsg;
     return false;
   }
   else {
@@ -593,7 +660,7 @@ bool MEPProvider::open_file() const
   while (!good) {
     // If looping on input is configured, do it
     if (m_current == m_connections.end()) {
-      if (m_config.non_stop) {
+      if (m_non_stop.value()) {
         m_current = m_connections.begin();
       }
       else {
@@ -609,7 +676,7 @@ bool MEPProvider::open_file() const
       good = true;
     }
     else {
-      m_logger->error(std::string{"Failed to open "} + *m_current + " " + strerror(errno));
+      error() << "Failed to open " << *m_current << " " << strerror(errno) << endmsg;
       m_read_error = true;
       return false;
     }
@@ -676,7 +743,7 @@ void MEPProvider::mep_read()
     // If we've been stopped, wait for start or exit
     if (!m_started || m_stopping) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
-      this->debug_output("Waiting for start", 0);
+      debug() << "Waiting for start" << endmsg;
       m_control_cond.wait(lock, [this] { return m_started || m_done; });
     }
 
@@ -706,7 +773,7 @@ void MEPProvider::mep_read()
       break;
     }
 
-    this->debug_output("Writing to MEP slice index " + std::to_string(i_buffer));
+    debug() << "Writing to MEP slice index " << i_buffer << endmsg;
 
     auto& read_buffer = m_read_buffers[i_buffer];
     auto& slice = m_net_slices[i_buffer];
@@ -736,7 +803,7 @@ void MEPProvider::mep_read()
         // Try to open the next file, if there is none, prefetching
         // is done.
         if (!m_read_error) {
-          this->debug_output("Prefetch done: eof and no more files");
+          debug() << "Prefetch done: eof and no more files" << endmsg;
         }
         receive_done = true;
         break;
@@ -769,11 +836,11 @@ void MEPProvider::mep_read()
       }
       if (receive_done) {
         m_done = receive_done;
-        this->debug_output("Prefetch notifying all");
+        if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying all" << endmsg;
         m_mpi_cond.notify_all();
       }
       else if (!eof) {
-        this->debug_output("Prefetch notifying one");
+        if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
         m_mpi_cond.notify_one();
       }
     }
@@ -785,31 +852,30 @@ void MEPProvider::mep_read()
 void MEPProvider::mpi_read()
 {
   #ifdef HAVE_MPI
-  int window_size = m_config.window_size;
-  std::vector<MPI_Request> requests(window_size);
+  std::vector<MPI_Request> requests(m_window_size);
 
   // Iterate over the slices
   size_t reporting_period = 5;
-  std::vector<std::tuple<size_t, size_t>> data_received(m_config.n_receivers());
-  std::vector<size_t> n_meps(m_config.n_receivers());
+  std::vector<std::tuple<size_t, size_t>> data_received(m_receivers.size());
+  std::vector<size_t> n_meps(m_receivers.size());
   Timer t;
   Timer t_origin;
-  bool error = false;
+  bool mpi_error = false;
 
-  for (size_t i = 0; i < m_config.n_receivers(); ++i) {
+  for (size_t i = 0; i < m_receivers.size(); ++i) {
     auto [mpi_rank, numa_domain] = m_domains[i];
     MPI_Recv(&n_meps[i], 1, MPI_SIZE_T, mpi_rank, MPI::message::number_of_meps, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
   }
   size_t number_of_meps = std::accumulate(n_meps.begin(), n_meps.end(), 0u);
 
   size_t current_mep = 0;
-  while (!m_done && (m_config.non_stop || current_mep < number_of_meps)) {
+  while (!m_done && (m_non_stop.value() || current_mep < number_of_meps)) {
     // info_cout << MPI::rank_str() << "round " << current_file << "\n";
 
     // If we've been stopped, wait for start or exit
     if (!m_started || m_stopping) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
-      this->debug_output("Waiting for start", 0);
+      debug() << "Waiting for start" << endmsg;
       m_control_cond.wait(lock, [this] { return m_started || m_done; });
     }
 
@@ -832,12 +898,13 @@ void MEPProvider::mpi_read()
       }
     }
 
-    auto receiver = i_buffer % m_config.n_receivers();
+    auto receiver = i_buffer % m_receivers.size();
     auto [sender_rank, numa_node] = m_domains[receiver];
 
-    this->debug_output(
-      "Receiving from rank " + std::to_string(sender_rank) + " into buffer " + std::to_string(i_buffer) +
-      "  NUMA domain " + std::to_string(numa_node));
+    if (msgLevel(MSG::DEBUG)) {
+      debug() << "Receiving from rank " << sender_rank << " into buffer " << i_buffer
+              << " NUMA domain " << numa_node << endmsg;
+    }
 
     auto& slice = m_net_slices[i_buffer];
     char*& contents = m_mpi_buffers[i_buffer];
@@ -866,8 +933,7 @@ void MEPProvider::mpi_read()
           m_topology, contents, slice.slice_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
         if (s != 0) {
           m_read_error = true;
-          m_logger->error(std::string{"Failed to bind memory to node "} + std::to_string(numa_node) + " " + strerror(errno));
-          break;
+          throw GaudiException{name(), "Failed to bind memory to node "s + std::to_string(numa_node) + " " + strerror(errno), StatusCode::FAILURE};
         }
       }
 
@@ -887,7 +953,7 @@ void MEPProvider::mpi_read()
     // Size of the last message (if the MFP size is not a multiple of MPI::mdf_chunk_size)
     int rest = mep_size - n_messages * MPI::mdf_chunk_size;
     // Number of parallel sends
-    int n_sends = n_messages > window_size ? window_size : n_messages;
+    int n_sends = n_messages > m_window_size.value() ? m_window_size.value() : n_messages;
 
     // info_cout << MPI::rank_str() << "n_messages " << n_messages << ", rest " << rest << ", n_sends " << n_sends <<
     // "\n";
@@ -907,7 +973,7 @@ void MEPProvider::mpi_read()
     // Sliding window sends
     for (int k = n_sends; k < n_messages; k++) {
       int r;
-      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      MPI_Waitany(m_window_size, requests.data(), &r, MPI_STATUS_IGNORE);
       char* message = contents + k * MPI::mdf_chunk_size;
       MPI_Irecv(
         message,
@@ -921,7 +987,7 @@ void MEPProvider::mpi_read()
     // Last send (if necessary)
     if (rest) {
       int r;
-      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      MPI_Waitany(m_window_size, requests.data(), &r, MPI_STATUS_IGNORE);
       char* message = contents + n_messages * MPI::mdf_chunk_size;
       MPI_Irecv(
         message,
@@ -954,7 +1020,7 @@ void MEPProvider::mpi_read()
       const auto seconds = t.get_elapsed_time();
       auto total_rate = 0.;
       auto total_bandwidth = 0.;
-      for (size_t i_rec = 0; i_rec < m_config.n_receivers(); ++i_rec) {
+      for (size_t i_rec = 0; i_rec < m_receivers.size(); ++i_rec) {
         auto& [mr, br] = data_received[i_rec];
         auto [rec_rank, rec_node] = m_domains[i_rec];
 
@@ -974,7 +1040,7 @@ void MEPProvider::mpi_read()
         br = 0;
         mr = 0;
       }
-      if (m_config.n_receivers() > 1) {
+      if (m_receivers.size() > 1) {
         printf(
           "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s\n",
           t_origin.get_elapsed_time(),
@@ -987,13 +1053,13 @@ void MEPProvider::mpi_read()
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
-    if (!error) {
+    if (!mpi_error) {
       {
         std::unique_lock<std::mutex> lock {m_mpi_mutex};
         set_intervals(m_buffer_status[i_buffer].intervals, size_t {slice.packing_factor});
         assert(m_buffer_status[i_buffer].work_counter == 0);
       }
-      this->debug_output("Prefetch notifying one");
+      if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
       m_mpi_cond.notify_one();
     }
     m_mpi_cond.notify_one();
@@ -1003,7 +1069,7 @@ void MEPProvider::mpi_read()
 
   if (!m_done) {
     m_done = true;
-    this->debug_output("Prefetch notifying all");
+    if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying all" << endmsg;
     m_mpi_cond.notify_all();
   }
   #endif
@@ -1015,8 +1081,8 @@ void MEPProvider::bm_read()
 {
   auto to_read = this->n_events();
   if (to_read)
-    m_logger->error("Number of events makes no sense when receiving from"
-                    " the buffer manager: ignoring");
+    error() << "Number of events makes no sense when receiving from"
+            << " the buffer manager: ignoring" << endmsg;
 
   while (!m_done) {
     // info_cout << MPI::rank_str() << "round " << current_file << "\n";
@@ -1024,7 +1090,7 @@ void MEPProvider::bm_read()
     // If we've been stopped, wait for start or exit
     if (!m_started || m_stopping) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
-      this->debug_output("Waiting for start", 0);
+      debug() << "Waiting for start" << endmsg;
       m_control_cond.wait(lock, [this] { return m_started || m_done; });
     }
 
@@ -1047,7 +1113,9 @@ void MEPProvider::bm_read()
       break;
     }
 
-    this->debug_output("Writing to MEP slice index " + std::to_string(i_buffer));
+    if (msgLevel(MSG::DEBUG)) {
+      debug() << "Writing to MEP slice index " << i_buffer << endmsg;
+    }
 
     auto& slice = m_net_slices[i_buffer];
 
@@ -1058,7 +1126,7 @@ void MEPProvider::bm_read()
       int ev_type = 0, *ev_data = 0;
       long ev_len = 0;
 
-      auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, m_config.partitionID);
+      auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, m_partitionID);
       if (sc == MBM_NORMAL)  {
         slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(ev_data);
         slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep_header->bytes()};
@@ -1066,7 +1134,7 @@ void MEPProvider::bm_read()
         auto const* block_header = MEP::block_header(slice.mep_header, 0);
         slice.packing_factor = block_header->n_banks;
 
-        m_logger->debug(std::string{"Got mep with packing factor "} + std::to_string(slice.packing_factor));
+        if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
       } else if (sc == MBM_REQ_CANCEL) {
         std::unique_lock<std::mutex> lock {m_mpi_mutex};
         m_buffer_status[i_buffer].writable = true;
@@ -1083,7 +1151,7 @@ void MEPProvider::bm_read()
       break;
     }
 
-    assert(slice.packing_factor = m_packing_factor);
+    assert(cancelled || slice.packing_factor == m_packing_factor);
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
@@ -1097,11 +1165,11 @@ void MEPProvider::bm_read()
     }
 
     if (m_done) {
-      this->debug_output("Prefetch notifying all");
+      debug() << "Prefetch notifying all" << endmsg;
       m_mpi_cond.notify_all();
     }
     else if (!cancelled) {
-      this->debug_output("Prefetch notifying one");
+      debug() << "Prefetch notifying one" << endmsg;
       m_mpi_cond.notify_one();
     }
   }
@@ -1147,21 +1215,21 @@ void MEPProvider::transpose(int thread_id)
       ++(status.work_counter);
       status.writable = false;
 
-      this->debug_output(
-        "Got MEP slice index " + std::to_string(i_buffer) + " interval [" + std::to_string(std::get<0>(interval)) +
-          "," + std::to_string(std::get<1>(interval)) + ")",
-        thread_id);
+      if (msgLevel(MSG::DEBUG)) {
+        debug() << "Transpose " << thread_id << ": Got MEP slice index " << i_buffer << " interval [" << std::get<0>(interval)
+                << "," << std::get<1>(interval) << ")" << endmsg;
+      }
     }
 
     // Get a slice to write to
     if (!slice_index) {
-      this->debug_output("Getting slice index", thread_id);
+      if (msgLevel(MSG::DEBUG)) debug() << "Transpose " << thread_id << ": Getting slice index" << endmsg;
       auto it = m_slice_free.end();
       {
         std::unique_lock<std::mutex> lock {m_slice_mut};
         it = find(m_slice_free.begin(), m_slice_free.end(), true);
         if (it == m_slice_free.end()) {
-          this->debug_output("Waiting for free slice", thread_id);
+          if (msgLevel(MSG::DEBUG)) debug() << "Transpose " << thread_id << ": Waiting for free slice" << endmsg;
           m_slice_cond.wait(lock, [this, &it] {
             it = std::find(m_slice_free.begin(), m_slice_free.end(), true);
             return it != m_slice_free.end() || m_transpose_done;
@@ -1175,7 +1243,7 @@ void MEPProvider::transpose(int thread_id)
         }
         *it = false;
         slice_index = distance(m_slice_free.begin(), it);
-        this->debug_output("Got slice index " + std::to_string(*slice_index), thread_id);
+        if (msgLevel(MSG::DEBUG)) debug() << "Transpose " << thread_id << ": Got slice index " << *slice_index << endmsg;
 
         // Keep track of what buffer this slice belonged to
         m_slice_to_buffer[*slice_index] = {i_buffer, std::get<0>(interval), std::get<1>(interval)};
@@ -1184,8 +1252,7 @@ void MEPProvider::transpose(int thread_id)
 
     // Reset the slice
     auto& event_ids = m_event_ids[*slice_index];
-    //    reset_slice(m_slices, *slice_index, bank_types, event_ids, !m_config.transpose_mep);
-    reset_slice(m_slices, *slice_index, types(), event_ids, !m_config.transpose_mep);
+    reset_slice(m_slices, *slice_index, types(), event_ids, !m_transpose_mep.value());
 
     // MEP data
     auto& slice = m_net_slices[i_buffer];
@@ -1197,29 +1264,33 @@ void MEPProvider::transpose(int thread_id)
     MEP::fragment_offsets(slice.blocks, slice.offsets);
 
     // Transpose or calculate offsets
-    if (m_config.transpose_mep) {
+    if (m_transpose_mep.value()) {
       // Transpose the events into the slice
       std::tie(good, transpose_full, n_transposed) = MEP::transpose_events(
         m_slices,
         *slice_index,
         m_bank_ids,
-        this->types(),
+        m_bank_types,
         m_banks_count,
         event_ids,
         slice.mep_header,
         slice.blocks,
         slice.offsets,
-        interval);
-      this->debug_output(
-        "Transposed slice " + std::to_string(*slice_index) + "; good: " + std::to_string(good) +
-          "; full: " + std::to_string(transpose_full) + "; n_transposed:  " + std::to_string(n_transposed),
-        thread_id);
+        interval,
+        m_split_by_run.value());
+      if (msgLevel(MSG::DEBUG)) {
+        debug() << "Transpose " << thread_id << ": Transposed slice " << *slice_index
+                << "; good: " << good << "; full: " << transpose_full
+                << "; n_transposed: " << n_transposed << endmsg;
+      }
     }
     else {
       // Calculate fragment offsets in MEP per sub-detector
       std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
         m_slices, *slice_index, m_bank_ids, this->types(), m_banks_count, event_ids, slice.blocks, interval);
-      this->debug_output("Calculated MEP offsets for slice " + std::to_string(*slice_index), thread_id);
+      if (msgLevel(MSG::DEBUG)) {
+        debug() << "Transpose " << thread_id << ": Calculated MEP offsets for slice " << *slice_index << endmsg;
+      }
     }
 
     if (m_read_error || !good) {
-- 
GitLab


From 735a9532cdb79275c906360ebee21d53c1c34a26 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:32:17 +0200
Subject: [PATCH 013/120] Fix MEP::find_blocks

---
 AllenOnline/AllenOnline/TransposeMEP.h | 2 +-
 AllenOnline/src/MEPProvider.cpp        | 1 +
 AllenOnline/src/TransposeMEP.cpp       | 9 ++++-----
 3 files changed, 6 insertions(+), 6 deletions(-)

diff --git a/AllenOnline/AllenOnline/TransposeMEP.h b/AllenOnline/AllenOnline/TransposeMEP.h
index 5691b4ed7..a925dd1cf 100644
--- a/AllenOnline/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/AllenOnline/TransposeMEP.h
@@ -60,7 +60,7 @@ namespace MEP {
   std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
   fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& data, std::vector<int> const& bank_ids);
 
-  void find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor, gsl::span<char const> const& buffer_span, Blocks& blocks);
+  void find_blocks(MEP::MEP_header const* mep_header, gsl::span<char const> const& buffer_span, Blocks& blocks);
 
   void fragment_offsets(Blocks const& blocks, std::vector<std::vector<uint32_t>>& offsets);
 
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 2ad214e8f..866e1a8fb 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -1259,6 +1259,7 @@ void MEPProvider::transpose(int thread_id)
 
     // Fill blocks
     MEP::find_blocks(slice.mep_header, slice.packing_factor, slice.mep_data, slice.blocks);
+    MEP::find_blocks(slice.mep_header, slice.mep_data, slice.blocks);
 
     // Fill fragment offsets
     MEP::fragment_offsets(slice.blocks, slice.offsets);
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index fb891a727..fab35e3c4 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -50,16 +50,15 @@ MEP::fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& mep
   return {true, count, versions};
 }
 
-void MEP::find_blocks(MEP::MEP_header const* mep_header, unsigned packing_factor, gsl::span<char const> const& buffer_span, Blocks& blocks)
+void MEP::find_blocks(MEP::MEP_header const* mep_header, gsl::span<char const> const& buffer_span, Blocks& blocks)
 {
   // Fill blocks
-  auto block_hdr_size = MFP::MFP_header_size(packing_factor, MFP::MFP_aligment);
-
   for (size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
     // block offsets are in 4-byte words with respect to the start of the MEP header
     auto block_offset = mep_header->offsets()[i_block] * sizeof(uint32_t);
-    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>(buffer_span.data() + block_offset);
-    gsl::span<char const> block_data {buffer_span.data() + block_offset + block_hdr_size, block_header->bytes() -  block_hdr_size};
+    char const* block_start = buffer_span.data() + block_offset;
+    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>(block_start);
+    gsl::span<char const> block_data {block_start + block_header->header_size(), block_header->bytes() -  block_header->header_size()};
     blocks[i_block] = std::tuple {block_header, std::move(block_data)};
   }
 }
-- 
GitLab


From 216c170944357ef1d63c9f154d9437ca4b95e957 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:33:22 +0200
Subject: [PATCH 014/120] Fix packing factor retrieval and extend mep reading
 example

---
 AllenOnline/application/read_mep.cpp | 73 ++++++++++++++++++++++------
 AllenOnline/src/MEPProvider.cpp      |  1 -
 AllenOnline/src/ReadMEP.cpp          |  2 +-
 3 files changed, 59 insertions(+), 17 deletions(-)

diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index a648a8083..f395bf1fa 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -18,12 +18,14 @@
 #include <Event/RawBank.h>
 
 #include <Allen/Logger.h>
+#include <Allen/SliceUtils.h>
 
 #include <MDF/StreamDescriptor.h>
 
 #include <EventBuilding/MEP_tools.hpp>
 #include <EventBuilding/MFP_tools.hpp>
 
+#include <AllenOnline/TransposeMEP.h>
 #include <ReadMEP.h>
 
 using namespace std;
@@ -40,8 +42,6 @@ int main(int argc, char* argv[])
 
   // Some storage for reading the events into
   bool eof = false, success = false;
-  MEP::MEP_header const* mep_header;
-  gsl::span<char const> mep_span;
 
   auto input = LHCb::StreamDescriptor::bind(filename);
   if (input.ioDesc != 0) {
@@ -53,33 +53,76 @@ int main(int argc, char* argv[])
   }
 
   vector<char> data;
-  unsigned packing_factor = 0;
 
-  size_t i_mep = 0;
-  while (!eof && i_mep++ < n_meps) {
+  EventIDs event_ids;
+  auto bank_ids = Allen::bank_ids();
 
-    std::tie(eof, success, mep_header, packing_factor, mep_span) = MEP::read_mep(input, data);
+  MEP::Slices mep_slices(1);
+  auto& slice = mep_slices[0];
+
+  Allen::Slices bank_slices;
+
+  std::unordered_set bank_types = {BankTypes::ODIN};
+
+  for (size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep) {
+
+    std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(input, data);
     if (!success) {
       return 1;
     }
-    for (uint16_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
-      // block offsets are in number of 4-byte words
-      auto const block_offset = mep_header->offsets()[i_block] * sizeof(uint32_t);
-      auto block_header = reinterpret_cast<MFP::MFP_header const*>(mep_span.data() + block_offset);
-      char const* block_data = static_cast<char const*>(block_header->payload());
-      [[maybe_unused]] char const* block_end = block_data + block_header->bytes() - block_header->header_size();
+    else {
+      cout << "Read mep with packing factor " << slice.packing_factor << "\n";
+    }
 
-      assert(static_cast<size_t>(block_end - mep_span.data()) <= mep_header->offsets()[i_block + 1] * sizeof(uint32_t));
+    if (i_mep == 0) {
+      event_ids.reserve(slice.packing_factor);
+      slice.blocks.resize(slice.mep_header->n_MFPs, MEP::Blocks::value_type{});
+      slice.offsets.resize(slice.mep_header->n_MFPs);
+      for (auto& offsets : slice.offsets) {
+        offsets.resize(slice.packing_factor + 1);
+      }
+    }
 
+    MEP::find_blocks(slice.mep_header, slice.mep_data, slice.blocks);
+    auto [sucess, banks_count, banks_version] = MEP::fill_counts(slice.mep_header, slice.mep_data, bank_ids);
+
+    if (i_mep == 0) {
+      bank_slices = allocate_slices(1, bank_types,
+        [pf = slice.packing_factor, &bank_ids, bc = banks_count](auto bt) -> std::tuple<size_t, size_t> {
+          auto it = std::find(bank_ids.begin(), bank_ids.end(), to_integral(bt));
+          auto lhcb_type = std::distance(bank_ids.begin(), it);
+          auto n_blocks = bc[lhcb_type];
+          return {0ul, 2 + n_blocks + (1 + pf) * (1 + n_blocks) - 2};
+        });
+    }
+    else {
+      reset_slice(bank_slices, 0, bank_types, event_ids, true);
+    }
+
+
+    for (auto const& [block_header, block_data] : slice.blocks) {
+      // block offsets are in number of 4-byte words
       auto lhcb_type = int {block_header->bank_types()[0]};
 
       cout << "fragment"
            << " packing: " << std::setw(4) << block_header->n_banks << " event_id: " << std::setw(6)
            << block_header->ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
            << (block_header->src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{block_header->block_version}
-           << " size: " << std::setw(6) << block_header->bytes() << "\n";
+           << " size: " << std::setw(8) << block_header->bytes() - block_header->header_size() << "\n";
     }
-  }
 
+    MEP::mep_offsets(bank_slices, 0,
+                     bank_ids,
+                     bank_types,
+                     banks_count,
+                     event_ids,
+                     slice.blocks,
+                     {0ul, slice.packing_factor},
+                     false);
+
+    for (size_t i = 0; i < std::min(10ul, event_ids.size()); ++i) {
+      cout << std::get<0>(event_ids[i]) << " " << std::get<1>(event_ids[i]) << "\n";
+    }
+  }
   return 0;
 }
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 866e1a8fb..d4b97e9cb 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -1258,7 +1258,6 @@ void MEPProvider::transpose(int thread_id)
     auto& slice = m_net_slices[i_buffer];
 
     // Fill blocks
-    MEP::find_blocks(slice.mep_header, slice.packing_factor, slice.mep_data, slice.blocks);
     MEP::find_blocks(slice.mep_header, slice.mep_data, slice.blocks);
 
     // Fill fragment offsets
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index a2a06ba82..25d7d0466 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -75,7 +75,7 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   }
 
   // Get the packing factor
-  auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(static_cast<char const*>(mep_header->payload()) + mep_header->offsets()[0] * sizeof(uint32_t));
+  auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(mep_buffer + mep_header->offsets()[0] * sizeof(uint32_t));
 
   return {false, true, mep_header, block_header->n_banks,
           {buffer.data(), data_size}};
-- 
GitLab


From a89474131a2c5a1e485507905a40cecf331a2d3a Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:40:05 +0200
Subject: [PATCH 015/120] Use AllenConfiguration to get partition parameters

---
 AllenOnline/AllenOnline/MEPProvider.h |  9 ++++-----
 AllenOnline/src/MEPProvider.cpp       | 28 +++++++++++++++++++--------
 2 files changed, 24 insertions(+), 13 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 900240729..344af35d8 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -53,6 +53,8 @@ namespace MEP {
                   MPI)
 }
 
+class AllenConfiguration;
+
 /**
  * @brief      Configuration parameters for the MEPProvider
  */
@@ -248,6 +250,8 @@ private:
   // Iterator that points to the filename of the currently open file
   mutable std::vector<std::string>::const_iterator m_current;
 
+  AllenConfiguration const* m_allenConfig;
+
   size_t n_buffers() const { return std::get<0>(m_bufferConfig.value()); }
 
   size_t n_receivers() const { return m_receivers.size(); }
@@ -267,7 +271,6 @@ private:
 
   std::unordered_set<BankTypes> m_bank_types;
 
-  Gaudi::Property<std::string> m_process_name {this, "ProcessName"};
   Gaudi::Property<size_t> m_nslices {this, "NSlices", 6};
   Gaudi::Property<size_t> m_events_per_slice {this, "EventsPerSlice", 1000};
   Gaudi::Property<std::vector<std::string>> m_connections {this, "Connections"};
@@ -283,10 +286,6 @@ private:
 
   // Mapping of receiver card to MPI rank to receive from
   Gaudi::Property<std::map<std::string, int>> m_receivers {this, "Receivers", {}};
-  Gaudi::Property<std::string> m_bufferName {this, "BufferName", "Events"};
-  Gaudi::Property<std::string> m_partition {this, "Partition", "Test"};
-  Gaudi::Property<unsigned> m_partitionID {this, "PartitionID", 0};
-  Gaudi::Property<bool> m_partitionBuffers {this, "PartitionBuffers", false};
   Gaudi::Property<std::vector<std::string>> m_requests {this, "Requests", {}};
 
 };
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index d4b97e9cb..0404c4bcf 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -20,6 +20,8 @@
 #include "AllenOnline/MEPProvider.h"
 #include "AllenOnline/TransposeMEP.h"
 
+#include "AllenConfiguration.h"
+
 #include "WriteMDF.h"
 #include "ReadMEP.h"
 
@@ -277,6 +279,13 @@ StatusCode MEPProvider::initialize()
   m_buffer_transpose = m_buffer_status.begin();
   m_buffer_reading = m_buffer_status.begin();
 
+  auto config = service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
+  if (!config) {
+    error() << "Failed to retrieve AllenConfiguration." << endmsg;
+    return StatusCode::FAILURE;
+  }
+  m_allenConfig = config.get();
+
   StatusCode sc = StatusCode::SUCCESS;
   if (m_source == MEP::ProviderSource::MPI) {
     sc = init_mpi();
@@ -525,21 +534,23 @@ StatusCode MEPProvider::init_mpi()
 
 int MEPProvider::init_bm()
 {
-  auto buffer_name = m_bufferName.value();
-  if ( m_partitionBuffers.value() ) {
+  auto buffer_name = m_connections.value().back();
+  auto const partition = m_allenConfig->partition.value();
+  auto const partitionID = m_allenConfig->partitionID.value();
+  if ( m_allenConfig->partitionBuffers.value() ) {
     std::stringstream stream;
-    stream << std::hex << m_partitionID.value();
+    stream << std::hex << partitionID;
     buffer_name += "_";
-    buffer_name += m_partition.value().empty() ? stream.str() : m_partition.value();
+    buffer_name += partition.empty() ? stream.str() : partition;
   }
 
   for (size_t b = 0; b < n_buffers(); ++b) {
-    auto pn = m_process_name + "." + std::to_string(b);
+    auto pn = RTL::processName() + "." + std::to_string(b);
     BMID bmid;
     if (b == 0) {
-      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), m_partitionID, BM_COM_FIFO);
+      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), partitionID, BM_COM_FIFO);
     } else {
-      bmid = ::mbm_connect(m_bmIDs[0], pn.c_str(), m_partitionID);
+      bmid = ::mbm_connect(m_bmIDs[0], pn.c_str(), partitionID);
     }
 
     if (bmid == MBM_INV_DESC)   {
@@ -1079,6 +1090,7 @@ void MEPProvider::mpi_read()
 // buffer manager reader thread
 void MEPProvider::bm_read()
 {
+  auto const partitionID = m_allenConfig->partitionID.value();
   auto to_read = this->n_events();
   if (to_read)
     error() << "Number of events makes no sense when receiving from"
@@ -1126,7 +1138,7 @@ void MEPProvider::bm_read()
       int ev_type = 0, *ev_data = 0;
       long ev_len = 0;
 
-      auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, m_partitionID);
+      auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, partitionID);
       if (sc == MBM_NORMAL)  {
         slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(ev_data);
         slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep_header->bytes()};
-- 
GitLab


From b9e096263450100790e6974a514aa92003889d69 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:41:46 +0200
Subject: [PATCH 016/120] Don't use bare pointer to AllenConfiguration

---
 AllenOnline/src/AllenApplication.cpp | 11 +++++------
 AllenOnline/src/AllenApplication.h   |  2 +-
 2 files changed, 6 insertions(+), 7 deletions(-)

diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 632813ebd..d37dd29cf 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -151,12 +151,11 @@ int AllenApplication::configureApplication()
     m_monSvc->declareInfo("OUT", m_monitor.eventsOut, "Number of events fully output", m_monEvents);
   }
 
-  auto config = sloc->service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
-  if (!config.get()) {
+  m_allenConfig = sloc->service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
+  if (!m_allenConfig) {
     m_logger->throwError("Failed to retrieve AllenConfiguration.");
     return Online::ONLINE_ERROR;
   }
-  m_allenConfig = config.get();
 
   m_zmqSvc = sloc->service<IZeroMQSvc>("ZeroMQSvc");
   if (!m_zmqSvc) {
@@ -310,12 +309,12 @@ void AllenApplication::allenLoop()
   //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
   //--device 23:00.0
   std::map<std::string, std::string> allen_options = {{"v", std::to_string(6 - m_config->outputLevel())},
-                                                      {"t", std::to_string(m_allenConfig->nThreads.value())},
+                                                      {"t", m_allenConfig->nThreads.toString()},
                                                       {"geometry", paramDir},
                                                       {"configuration", m_json},
                                                       {"device", m_allenConfig->device.value()},
-                                                      {"s", std::to_string(m_nSlices)}};
-
+                                                      {"s", std::to_string(m_nSlices)},
+                                                      {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())}};
   allen(allen_options, m_updater, m_provider, m_output.get(), m_zmqSvc.get(), m_controlConnection);
 }
 
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index f59b80557..6f48cf321 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -80,7 +80,7 @@ private:
   SmartIF<IZeroMQSvc> m_zmqSvc;
 
   Allen::NonEventData::IUpdater* m_updater = nullptr;
-  AllenConfiguration const* m_allenConfig = nullptr;
+  SmartIF<AllenConfiguration> m_allenConfig;
 
   std::string m_controlConnection = "inproc://AllenApplicationControl";
 
-- 
GitLab


From 9ba7f7f86c061d4db7282e4460a9b4901614a9be Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:42:33 +0200
Subject: [PATCH 017/120] Remove unneeded property

---
 AllenOnline/src/AllenConfiguration.h | 1 -
 1 file changed, 1 deletion(-)

diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index 078dae923..d5c4923a0 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -26,7 +26,6 @@ public:
   Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json"};
   Gaudi::Property<size_t> nLines{this, "NLines", 0ul};
   Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
-  Gaudi::Property<std::vector<std::string>> input {this, "Input"};
 
   Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
   Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
-- 
GitLab


From 367d5cab959b79233d4340f53ef9a497455a33b2 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:46:33 +0200
Subject: [PATCH 018/120] Fix raw MEP slice allocation

---
 AllenOnline/src/MEPProvider.cpp | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 0404c4bcf..eff04e230 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -291,10 +291,10 @@ StatusCode MEPProvider::initialize()
     sc = init_mpi();
     if (!sc.isSuccess()) return sc;
   }
-  else {
+  else if (m_source == MEP::ProviderSource::Files) {
     m_read_buffers.resize(n_buffers());
-    m_net_slices.resize(n_buffers());
   }
+  m_net_slices.resize(n_buffers());
 
   // Initialize the current input filename
   m_current = m_connections.begin();
@@ -516,13 +516,13 @@ StatusCode MEPProvider::init_mpi()
     }
 
     Allen::host_register(contents, n_bytes, Allen::hostRegisterDefault);
-    m_net_slices.emplace_back(
-      MEP::Slice{nullptr,
-                 0u,
-                 gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
-                 MEP::Blocks {},
-                 MEP::SourceOffsets {},
-                 n_bytes});
+    m_net_slices[i] = {
+      nullptr,
+      0u,
+      gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
+      MEP::Blocks {},
+      MEP::SourceOffsets {},
+      n_bytes};
     m_mpi_buffers.emplace_back(contents);
   }
   return StatusCode::SUCCESS;
-- 
GitLab


From f9f288f8fd5140dff4c38879b848cf4df62a4f43 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:47:47 +0200
Subject: [PATCH 019/120] Improve size_fun for slice allocation in MEPProvider

---
 AllenOnline/src/MEPProvider.cpp | 24 +++++++++++++++++++++++-
 1 file changed, 23 insertions(+), 1 deletion(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index eff04e230..3a24ed56e 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -646,7 +646,29 @@ bool MEPProvider::allocate_storage(size_t i_read)
     }
     else {
       auto aps = eps < 100 ? 100 : eps;
-      return {std::lround(it->second * aps * bank_size_fudge_factor * kB), eps};
+      // Lookup LHCb bank type corresponding to Allen bank type
+      auto type_it =
+        std::find_if(Allen::bank_types.begin(), Allen::bank_types.end(), [bank_type](const auto& entry) {
+          return entry.second == bank_type;
+        });
+      if (type_it == Allen::bank_types.end()) {
+        throw std::out_of_range {std::string {"Failed to lookup LHCb type for bank type "} + std::to_string(ib)};
+      }
+      auto lhcb_type = to_integral<LHCb::RawBank::BankType>(type_it->first);
+
+      // When events are transposed from the read buffer into
+      // the per-rawbank-type slices, a check is made each time
+      // to see if there is enough space available in a slice.
+      // To avoid having to read every event twice to get the
+      // size of all the banks, the size of the entire event is
+      // used for the check - 65 kB on average. To avoid
+      // problems for banks with very low average size like the
+      // ODIN bank - 0.1 kB, a fixed amount is also added.
+      auto n_bytes = std::lround(
+        ((1 + m_banks_count[lhcb_type]) * sizeof(uint32_t) + it->second * kB) * aps *
+          bank_size_fudge_factor +
+        2 * MB);
+      return {n_bytes, eps};
     }
   };
   m_slices = allocate_slices(m_nslices, types(), size_fun);
-- 
GitLab


From 4bec0e021e2e7797cb2ad08b48eda82859831828 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:50:44 +0200
Subject: [PATCH 020/120] Use our own properties directly

---
 AllenOnline/src/MEPProvider.cpp | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 3a24ed56e..6b6536383 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -608,7 +608,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
     assert(slice.packing_factor == m_packing_factor);
   }
 
-  auto eps = this->events_per_slice();
+  size_t const eps = m_events_per_slice.value();
   auto n_interval = m_packing_factor / eps;
   auto rest = m_packing_factor % eps;
   for (auto& s : m_buffer_status) {
@@ -671,7 +671,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
       return {n_bytes, eps};
     }
   };
-  m_slices = allocate_slices(m_nslices, types(), size_fun);
+  m_slices = allocate_slices(m_nslices, m_bank_types, size_fun);
 
   m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(m_nslices, std::tuple{-1, 0ul, 0ul});
 
@@ -748,7 +748,7 @@ std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvi
 void MEPProvider::set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events)
 {
   if (n_events == 0) return;
-  const auto eps = this->events_per_slice();
+  size_t const eps = m_events_per_slice.value();
   auto n_interval = n_events / eps;
   auto rest = n_events % eps;
   if (rest) {
@@ -1286,7 +1286,7 @@ void MEPProvider::transpose(int thread_id)
 
     // Reset the slice
     auto& event_ids = m_event_ids[*slice_index];
-    reset_slice(m_slices, *slice_index, types(), event_ids, !m_transpose_mep.value());
+    reset_slice(m_slices, *slice_index, m_bank_types, event_ids, !m_transpose_mep.value());
 
     // MEP data
     auto& slice = m_net_slices[i_buffer];
@@ -1321,7 +1321,7 @@ void MEPProvider::transpose(int thread_id)
     else {
       // Calculate fragment offsets in MEP per sub-detector
       std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
-        m_slices, *slice_index, m_bank_ids, this->types(), m_banks_count, event_ids, slice.blocks, interval);
+        m_slices, *slice_index, m_bank_ids, m_bank_types, m_banks_count, event_ids, slice.blocks, interval);
       if (msgLevel(MSG::DEBUG)) {
         debug() << "Transpose " << thread_id << ": Calculated MEP offsets for slice " << *slice_index << endmsg;
       }
-- 
GitLab


From 090f6504c41ca4ebcd438de3a2d4fdaa025a06f3 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:52:05 +0200
Subject: [PATCH 021/120] Move updateHandlers to header and invoke them
 immediately

---
 AllenOnline/AllenOnline/MEPProvider.h | 40 +++++++++++++++++++----
 AllenOnline/src/MEPProvider.cpp       | 46 ++++++++-------------------
 2 files changed, 48 insertions(+), 38 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 344af35d8..e21e6a655 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -252,12 +252,14 @@ private:
 
   AllenConfiguration const* m_allenConfig;
 
+  // MBM variables
+  std::vector<BMID> m_bmIDs;
+
   size_t n_buffers() const { return std::get<0>(m_bufferConfig.value()); }
 
   size_t n_receivers() const { return m_receivers.size(); }
 
-  // MBM variables
-  std::vector<BMID> m_bmIDs;
+  std::unordered_set<BankTypes> m_bank_types;
 
   Gaudi::Property<std::set<LHCb::RawBank::BankType>>
   m_lhcb_bank_types{this, "BankTypes",
@@ -267,9 +269,19 @@ private:
                      LHCb::RawBank::Muon,
                      LHCb::RawBank::ODIN,
                      LHCb::RawBank::EcalPacked,
-                     LHCb::RawBank::HcalPacked}};
-
-  std::unordered_set<BankTypes> m_bank_types;
+                     LHCb::RawBank::HcalPacked},
+    [this](auto &) -> void {
+      for (auto bt : m_lhcb_bank_types) {
+        auto it = Allen::bank_types.find(bt);
+        if (it == Allen::bank_types.end()) {
+          throw GaudiException(this->name(), "No Allen bank type defined for requested bank type "s
+                               + LHCb::RawBank::typeName(bt), StatusCode::FAILURE);
+        }
+        else {
+          m_bank_types.insert(it->second);
+        }
+      }
+    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
 
   Gaudi::Property<size_t> m_nslices {this, "NSlices", 6};
   Gaudi::Property<size_t> m_events_per_slice {this, "EventsPerSlice", 1000};
@@ -278,7 +290,23 @@ private:
   Gaudi::Property<MEP::ProviderSource> m_source {this, "Source", MEP::ProviderSource::MBM};
 
   // number of prefetch buffers and transpose threads
-  Gaudi::Property<std::pair<int, int>> m_bufferConfig {this, "BufferConfig", {8, 5}};
+  Gaudi::Property<std::pair<int, int>> m_bufferConfig {this, "BufferConfig", {8, 5},
+    [this](auto &) -> void {
+      // Sanity check on the number of buffers and threads
+      auto [n_buf, n_transpose] = m_bufferConfig.value();
+      if (n_buf < 2) {
+        warning() << "Too few read buffers requested, setting it to 2" << endmsg;
+        n_buf = 2;
+      }
+
+      if (n_transpose > n_buf - 1) {
+        warning() << "Too many transpose threads requested with respect "
+                  << "to the number of read buffers; reducing the number of threads to "
+                  << n_buf - 1 << endmsg;
+        n_transpose = n_buf - 1;
+      }
+      m_bufferConfig.set({n_buf, n_transpose});
+    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
   Gaudi::Property<int> m_window_size {this, "MPIWindow", 4};
   Gaudi::Property<bool> m_non_stop {this, "LoopOnMEPs", false};
   Gaudi::Property<bool> m_transpose_mep {this, "TransposeMEPs", false};
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 6b6536383..0ccee0fa6 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -54,38 +54,6 @@ MEPProvider::MEPProvider(std::string name, ISvcLocator* loc)
   : Service {name, loc},
     m_banks_count {0}
 {
-
-  m_bufferConfig.declareUpdateHandler([this](auto &) -> void {
-    // Sanity check on the number of buffers and threads
-    auto [n_buf, n_transpose] = m_bufferConfig.value();
-    if (n_buf < 2) {
-      warning() << "Too few read buffers requested, setting it to 2" << endmsg;
-      n_buf = 2;
-    }
-
-    if (n_transpose > n_buf - 1) {
-      warning() << "Too many transpose threads requested with respect "
-                << "to the number of read buffers; reducing the number of threads to "
-                << n_buf - 1 << endmsg;
-      n_transpose = n_buf - 1;
-    }
-    m_bufferConfig.set({n_buf, n_transpose});
-  });
-
-  m_lhcb_bank_types.declareUpdateHandler([this](auto &) -> void {
-    for (auto bt : m_lhcb_bank_types) {
-      auto it = Allen::bank_types.find(bt);
-      if (it == Allen::bank_types.end()) {
-        throw GaudiException(this->name(), "No Allen bank type defined for requested bank type "s
-                             + LHCb::RawBank::typeName(bt), StatusCode::FAILURE);
-      }
-      else {
-        m_bank_types.insert(it->second);
-      }
-    }
-  });
-
-
 }
 
 EventIDs MEPProvider::event_ids(size_t slice_index, std::optional<size_t> first, std::optional<size_t> last) const
@@ -97,6 +65,8 @@ EventIDs MEPProvider::event_ids(size_t slice_index, std::optional<size_t> first,
 BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) const
 {
   auto ib = to_integral<BankTypes>(bank_type);
+  assert(static_cast<size_t>(ib) < m_slices.size());
+  assert(slice_index < m_slices[ib].size());
   auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
 
   BanksAndOffsets bno;
@@ -266,6 +236,18 @@ StatusCode MEPProvider::initialize()
 
   m_buffer_status.resize(n_buffers());
 
+  if (m_bank_types.empty()) {
+    error() << "No bank types specified" << endmsg;
+    return StatusCode::FAILURE;
+  }
+  else if (m_lhcb_bank_types.size() != m_bank_types.size()) {
+    error() << "Not all LHCb bank types converted to Allen bank types" << endmsg;
+    return StatusCode::FAILURE;
+  }
+  else {
+    info() << "Providing banks for " << m_lhcb_bank_types.toString() << endmsg;
+  }
+
   init_input(m_nslices, m_events_per_slice, m_bank_types,
              m_transpose_mep.value() ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, {});
 
-- 
GitLab


From ad0913facf2fb4049b2d1c61211d30be10f70e6a Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:52:46 +0200
Subject: [PATCH 022/120] Use Gaudi logging

---
 AllenOnline/src/MEPProvider.cpp | 28 +++++++++++-----------------
 1 file changed, 11 insertions(+), 17 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 0ccee0fa6..7995f33f8 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -603,7 +603,6 @@ bool MEPProvider::allocate_storage(size_t i_read)
     slice.blocks.resize(n_blocks);
     slice.offsets.resize(n_blocks);
     for (auto& offsets : slice.offsets) {
-      // info_cout << "Packing factor: " << mep_header->packing_factor << "\n";
       offsets.resize(m_packing_factor + 1);
     }
   }
@@ -687,7 +686,7 @@ bool MEPProvider::open_file() const
 
     m_input = LHCb::StreamDescriptor::bind(*m_current);
     if (m_input->ioDesc != 0) {
-      info_cout << "Opened " << *m_current << "\n";
+      info() << "Opened " << *m_current << endmsg;
       good = true;
     }
     else {
@@ -734,11 +733,15 @@ void MEPProvider::set_intervals(std::vector<std::tuple<size_t, size_t>>& interva
   auto n_interval = n_events / eps;
   auto rest = n_events % eps;
   if (rest) {
-    debug_cout << "Set interval (rest): " << n_interval * eps << "," << n_interval * eps + rest << "\n";
+    if (msgLevel(MSG::DEBUG)) {
+      debug() << "Set interval (rest): " << n_interval * eps << "," << n_interval * eps + rest << endmsg;
+    }
     intervals.emplace_back(n_interval * eps, n_interval * eps + rest);
   }
   for (size_t i = n_interval; i != 0; --i) {
-    debug_cout << "Set interval: " << (i - 1) * eps << "," << i * eps << "\n";
+    if (msgLevel(MSG::DEBUG)) {
+      debug() << "Set interval: " << (i - 1) * eps << "," << i * eps << endmsg;
+    }
     intervals.emplace_back((i - 1) * eps, i * eps);
   }
 }
@@ -749,12 +752,10 @@ void MEPProvider::mep_read()
   bool receive_done = false;
 
   auto to_read = this->n_events();
-  if (to_read) debug_cout << "Reading " << *to_read << " events\n";
+  if (to_read && msgLevel(MSG::DEBUG)) debug() << "Reading " << *to_read << " events" << endmsg;
   auto to_publish = 0;
 
   while (!receive_done) {
-    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
-
     // If we've been stopped, wait for start or exit
     if (!m_started || m_stopping) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
@@ -788,7 +789,7 @@ void MEPProvider::mep_read()
       break;
     }
 
-    debug() << "Writing to MEP slice index " << i_buffer << endmsg;
+    if (msgLevel(MSG::DEBUG)) debug() << "Writing to MEP slice index " << i_buffer << endmsg;
 
     auto& read_buffer = m_read_buffers[i_buffer];
     auto& slice = m_net_slices[i_buffer];
@@ -799,7 +800,7 @@ void MEPProvider::mep_read()
       std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
 
       if (!eof) {
-        debug_cout << "Read mep with packing factor " << slice.packing_factor << "\n";
+        if (msgLevel(MSG::DEBUG)) debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
         if (to_read && success) {
           to_publish = std::min(*to_read, size_t {slice.packing_factor});
           *to_read -= to_publish;
@@ -817,7 +818,7 @@ void MEPProvider::mep_read()
       else if ((to_read && *to_read == 0) || (eof && !open_file())) {
         // Try to open the next file, if there is none, prefetching
         // is done.
-        if (!m_read_error) {
+        if (!m_read_error && msgLevel(MSG::DEBUG)) {
           debug() << "Prefetch done: eof and no more files" << endmsg;
         }
         receive_done = true;
@@ -885,8 +886,6 @@ void MEPProvider::mpi_read()
 
   size_t current_mep = 0;
   while (!m_done && (m_non_stop.value() || current_mep < number_of_meps)) {
-    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
-
     // If we've been stopped, wait for start or exit
     if (!m_started || m_stopping) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
@@ -970,9 +969,6 @@ void MEPProvider::mpi_read()
     // Number of parallel sends
     int n_sends = n_messages > m_window_size.value() ? m_window_size.value() : n_messages;
 
-    // info_cout << MPI::rank_str() << "n_messages " << n_messages << ", rest " << rest << ", n_sends " << n_sends <<
-    // "\n";
-
     // Initial parallel sends
     for (int k = 0; k < n_sends; k++) {
       char* message = contents + k * MPI::mdf_chunk_size;
@@ -1101,8 +1097,6 @@ void MEPProvider::bm_read()
             << " the buffer manager: ignoring" << endmsg;
 
   while (!m_done) {
-    // info_cout << MPI::rank_str() << "round " << current_file << "\n";
-
     // If we've been stopped, wait for start or exit
     if (!m_started || m_stopping) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
-- 
GitLab


From ab3992c30c5db268b0dbe19bd2eb1a74702c0bbe Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 11 Jun 2021 14:53:15 +0200
Subject: [PATCH 023/120] Update Allen.py with MEPProvider configuration

---
 AllenOnline/options/Allen.py | 108 ++++++++++++++++++++++++++++-------
 1 file changed, 87 insertions(+), 21 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 4afc546d0..dc47eed94 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -1,4 +1,6 @@
-#!/usr/bin/env python2
+#!/usr/bin/env python3
+import os
+import json
 from Configurables import LHCbApp, CondDB, ApplicationMgr
 from Configurables import DumpUTGeometry, DumpFTGeometry, DumpMuonTable
 from Configurables import DumpMuonGeometry, DumpVPGeometry, AllenUpdater
@@ -6,18 +8,41 @@ from Configurables import DumpMagneticField, DumpBeamline, DumpUTLookupTables
 from Configurables import AllenConfiguration
 from Configurables import MonitorSvc
 from Configurables import Online__Configuration as OnlineConfiguration
+from Configurables import MEPProvider
 from Allen.config import setup_allen_non_event_data_service
 
+try:
+    import OnlineEnvBase as OnlineEnv
+    run_online = True
+    output_level = OnlineEnv.OutputLevel
+    partition = OnlineEnv.PartitionName
+    partition_id = OnlineEnv.PartitionID
+except ImportError:
+    run_online = False
+    output_level = 3
+    partition = 'Allen'
+    partition_id = 0
+
+integration_test = False
+
 app = LHCbApp(
     DataType="Upgrade",
-    EvtMax=1000,
     Simulation=True,
-    DDDBtag="dddb-20171122",
-    CondDBtag="sim-20180530-vc-md100")
+    DDDBtag="dddb-20210218",
+    CondDBtag="sim-20201218-vc-md100")
 
 # Upgrade DBs
 CondDB().Upgrade = True
 
+# Get the number of lines from the configuration. This should probably
+# be done internally to Allen and retrieved somehow
+n_lines = 0
+json_file = "/daqarea1/fest/allen/configuration/config.json"
+with open(json_file) as jf:
+    json_config = json.load(jf)
+    gs = json_config.get('gather_selections', {})
+    active_lines = gs.get('names_of_active_lines', '')
+    n_lines = len(active_lines.split(',')) if active_lines else 0
 
 online_conf = OnlineConfiguration("Application")
 online_conf.debug = False
@@ -26,30 +51,71 @@ online_conf.autoStart = False
 online_conf.monitorType = 'MonitorSvc'
 online_conf.logDeviceType = 'RTL::Logger::LogDevice'
 online_conf.logDeviceFormat = '%TIME%LEVEL%-8NODE: %-32PROCESS %-20SOURCE'
-online_conf.OutputLevel = 3
-online_conf.IOOutputLevel = 3
+online_conf.OutputLevel = 1
+online_conf.IOOutputLevel = output_level
+
+if run_online:
+    import _fifo_log
+    _fifo_log.logger_set_tag(partition)
+    _fifo_log.logger_start()
+
 
 allen_conf = AllenConfiguration()
-allen_conf.EventsPerSlice = 1000
-allen_conf.NonStop = True
-allen_conf.MPI = False
-allen_conf.JSON = "/daqarea1/fest/allen/configuration/config.json"
-allen_conf.Receivers = {"mlx5_0": 1}
-allen_conf.NThreads = 8
-allen_conf.NSlices = 16
-# allen_conf.Output = "tcp://192.168.1.101:35000"
+allen_conf.StopTimeout = 5.
+allen_conf.NThreads = 1
+# Device is a string so the PCI ID can also be given
 # allen_conf.Device = "01:00.0"
-allen_conf.Input = [
-    "/scratch/raaij/mep/upgrade_mc_minbias_scifi_v5_pf3000.mep"
-]
 allen_conf.Device = "0"
-allen_conf.OutputLevel = 2
+allen_conf.JSON = json_file
+allen_conf.NLines = n_lines
+allen_conf.OutputChecksum = False
+allen_conf.ParamDir = "/daqarea1/fest/allen/fest_geometry"
+allen_conf.Partition = partition
+allen_conf.PartitionBuffers = True
+allen_conf.PartitionID = partition_id
+allen_conf.EnableRunChanges = False
+
+if run_online:
+    allen_conf.Output = "mbm://Output"
+elif integration_test:
+    allen_conf.Output = "tcp://192.168.1.101:35000"
+else:
+    allen_conf.Output = "allen_output.mdf"
+
+mep_provider = MEPProvider()
+mep_provider.NSlices = 8
+mep_provider.EventsPerSlice = 1000
+mep_provider.OutputLevel = 2
+# Number of MEP buffers and number of transpose/offset threads
+mep_provider.BufferConfig = (4, 3)
+mep_provider.TransposeMEPs = True
+mep_provider.SplitByRun = False
+
+if run_online:
+    mep_provider.Source = "MBM"
+    mep_provider.Connections = ["Input"]
+    mep_provider.Requests = [
+        'EvType=3;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
+    ]
+elif integration_test:
+    mep_provider.Source = "MPI"
+    mep_provider.Receivers = {"mlx5_0": 1}
+    mep_provider.LoopOnMEPs = True
+else:
+    mep_provider.Source = "Files"
+    mep_dir = "/daqarea1/fest/mep"
+    mep_provider.Connections = sorted([os.path.join(mep_dir, mep_file)
+                                       for mep_file in os.listdir(mep_dir)
+                                       if mep_file.endswith('.mep')])
+    mep_provider.LoopOnMEPs = False
+
 
 monSvc = MonitorSvc('MonitorSvc')
-monSvc.PartitionName = 'Allen'
 monSvc.ExpandNameInfix = '<proc>'
 monSvc.ExpandCounterServices = True
 monSvc.UniqueServiceNames = True
+monSvc.PartitionName = partition
+
 
 setup_allen_non_event_data_service()
 
@@ -62,7 +128,7 @@ ApplicationMgr().ExtSvc += [
 ApplicationMgr().EvtSel = "NONE"
 ApplicationMgr().ExtSvc += [
     'ToolSvc', 'AuditorSvc', 'AllenConfiguration',
-    'Online::Configuration/Application', 'ZeroMQSvc'
+    'Online::Configuration/Application', 'ZeroMQSvc',
+    'MEPProvider'
 ]
 
-# gaudi = AppMgr()
-- 
GitLab


From 7c2d716607695c1cd888cb8317ffd6dd08bdec36 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 14 Jun 2021 23:10:42 +0200
Subject: [PATCH 024/120] Ask buffermanager for MEPs

---
 AllenOnline/options/Allen.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index dc47eed94..eb74bdedb 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -95,7 +95,7 @@ if run_online:
     mep_provider.Source = "MBM"
     mep_provider.Connections = ["Input"]
     mep_provider.Requests = [
-        'EvType=3;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
+        'EvType=1;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
     ]
 elif integration_test:
     mep_provider.Source = "MPI"
-- 
GitLab


From 52ee2d0dd3ab8bd254a5d0957f2fb2a2c31363c8 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 14 Jun 2021 23:11:13 +0200
Subject: [PATCH 025/120] configure MessageSvc

---
 AllenOnline/options/Allen.py | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index eb74bdedb..6c6e632a1 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -8,7 +8,7 @@ from Configurables import DumpMagneticField, DumpBeamline, DumpUTLookupTables
 from Configurables import AllenConfiguration
 from Configurables import MonitorSvc
 from Configurables import Online__Configuration as OnlineConfiguration
-from Configurables import MEPProvider
+from Configurables import MEPProvider, MessageSvc
 from Allen.config import setup_allen_non_event_data_service
 
 try:
@@ -119,8 +119,17 @@ monSvc.PartitionName = partition
 
 setup_allen_non_event_data_service()
 
+app = ApplicationMgr()
+
+app.MessageSvcType = 'MessageSvc'
+app.OutputLevel = output_level
+messageSvc = MessageSvc('MessageSvc')
+messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
+messageSvc.OutputLevel = output_level
+
+
 # Add the services that will produce the non-event-data
-ApplicationMgr().ExtSvc += [
+app.ExtSvc += [
     monSvc
 ]
 
-- 
GitLab


From 9ca329fb3666bacc9375a908eeb5734cfc0a5eb2 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 14 Jun 2021 23:12:25 +0200
Subject: [PATCH 026/120] Use a single input thread for MBM connection; fix MBM
 read loop; Skip host registration for now.

---
 AllenOnline/AllenOnline/MEPProvider.h |   2 +-
 AllenOnline/src/MEPProvider.cpp       | 100 +++++++++++++-------------
 2 files changed, 53 insertions(+), 49 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index e21e6a655..519449ec1 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -207,7 +207,7 @@ private:
   std::vector<BufferStatus> m_buffer_status;
   std::vector<BufferStatus>::iterator m_buffer_transpose;
   std::vector<BufferStatus>::iterator m_buffer_reading;
-  std::vector<std::thread> m_input_threads;
+  std::optional<std::thread> m_input_thread;
 
   // Atomics to flag errors and completion
   std::atomic<bool> m_done = false;
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 7995f33f8..6c18c8a09 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -276,6 +276,9 @@ StatusCode MEPProvider::initialize()
   else if (m_source == MEP::ProviderSource::Files) {
     m_read_buffers.resize(n_buffers());
   }
+  else {
+    m_bmIDs.resize(n_buffers());
+  }
   m_net_slices.resize(n_buffers());
 
   // Initialize the current input filename
@@ -306,24 +309,22 @@ StatusCode MEPProvider::start()
   }
 
   // start MPI receive, MEP reading thread or BM thread
-  if (m_source == MEP::ProviderSource::MPI && m_input_threads.empty()) {
+  if (m_source == MEP::ProviderSource::MPI && !m_input_thread) {
 #ifdef HAVE_MPI
-    m_input_threads.emplace_back(&MEPProvider::mpi_read, this);
+    m_input_thread = std::thread{&MEPProvider::mpi_read, this};
 #else
     throw StrException {"MPI requested, but no MPI support built in."};
 #endif
   }
-  else if (m_source == MEP::ProviderSource::Files  && m_input_threads.empty()) {
-    m_input_threads.emplace_back(&MEPProvider::mep_read, this);
+  else if (m_source == MEP::ProviderSource::Files && !m_input_thread) {
+    m_input_thread = std::thread{&MEPProvider::mep_read, this};
   }
   else if (m_source == MEP::ProviderSource::MBM) {
     auto sc = init_bm();
     if (sc != MBM_NORMAL) {
       return StatusCode::FAILURE;
     }
-    for (size_t b = 0; b < n_buffers(); ++b) {
-      m_input_threads.emplace_back(&MEPProvider::bm_read, this);
-    }
+    m_input_thread = std::thread{&MEPProvider::bm_read, this};
   }
 
   std::unique_lock<std::mutex> lock {m_control_mutex};
@@ -345,15 +346,12 @@ StatusCode MEPProvider::stop()
 
   if (m_source == MEP::ProviderSource::MBM) {
     // Cancel all requests to the buffer manager for those who are waiting
-    std::unique_lock<std::mutex> lock {m_mpi_mutex};
     for (size_t b = 0; b < m_buffer_status.size(); ++b) {
       if (!m_buffer_status[b].writable) ::mbm_cancel_request(m_bmIDs[b]);
     }
 
-    for (auto& thread : m_input_threads) {
-      thread.join();
-    }
-    m_input_threads.clear();
+    m_input_thread->join();
+    m_input_thread.reset();
 
     for (auto bmid : m_bmIDs) {
       if (bmid != MBM_INV_DESC) {
@@ -538,18 +536,23 @@ int MEPProvider::init_bm()
     if (bmid == MBM_INV_DESC)   {
       error() << "MBM: Failed to connect to MBM buffer " << buffer_name << endmsg;
       return MBM_ERROR;
-    } else if (b == 0) {
-      // register buffer manager memory with the device runtime
-      char const* mem = nullptr;
-      size_t mem_size = 0;
-      auto sc = mbm_buffer_memory(bmid, &mem, &mem_size);
-      if (sc == MBM_NORMAL) {
-        Allen::host_register(const_cast<char*>(mem), mem_size, Allen::hostRegisterDefault);
-      } else {
-        error() << "MBM: Failed to obtain buffer start and size, MBM" << endmsg;
-        return MBM_ERROR;
-      }
     }
+    // else if (b == 0) {
+    //   // register buffer manager memory with the device runtime
+    //   char const* mem = nullptr;
+    //   size_t mem_size = 0;
+    //   auto sc = mbm_buffer_memory(bmid, &mem, &mem_size);
+    //   if (sc == MBM_NORMAL) {
+    //     info() << "MBM mem size: " << mem_size << endmsg;
+    //     // Allen::host_register(const_cast<char*>(mem), mem_size, Allen::hostRegisterDefault);
+    //   } else {
+    //     error() << "MBM: Failed to obtain buffer start and size, MBM" << endmsg;
+    //     return MBM_ERROR;
+    //   }
+    // }
+    using namespace std::chrono_literals;
+    std::this_thread::sleep_for(50ms);
+
     for(auto r : m_requests) {
       MBM::Requirement rq{r};
       int sc = ::mbm_add_req(bmid, rq.evtype, rq.trmask, rq.vetomask, rq.maskType,
@@ -559,7 +562,8 @@ int MEPProvider::init_bm()
         return MBM_ERROR;
       }
     }
-    m_bmIDs.emplace_back(bmid);
+    m_bmIDs[b] = bmid;
+    info() << "Buffer " << b << " " << m_bmIDs[b] << endmsg;
   }
   return MBM_NORMAL;
 }
@@ -831,7 +835,7 @@ void MEPProvider::mep_read()
       break;
     }
 
-    assert(slice.packing_factor = m_packing_factor);
+    assert(slice.packing_factor <= m_packing_factor);
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
@@ -1131,28 +1135,28 @@ void MEPProvider::bm_read()
 
     bool cancelled = false;
 
-    while (!cancelled) {
-      unsigned int trmask[BM_MASK_SIZE];
-      int ev_type = 0, *ev_data = 0;
-      long ev_len = 0;
-
-      auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, partitionID);
-      if (sc == MBM_NORMAL)  {
-        slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(ev_data);
-        slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep_header->bytes()};
-        slice.slice_size = static_cast<size_t>(ev_len);
-        auto const* block_header = MEP::block_header(slice.mep_header, 0);
-        slice.packing_factor = block_header->n_banks;
-
-        if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
-      } else if (sc == MBM_REQ_CANCEL) {
-        std::unique_lock<std::mutex> lock {m_mpi_mutex};
-        m_buffer_status[i_buffer].writable = true;
-        cancelled = true;
-        if (count_writable() == 0) {
-          m_done = true;
-          break;
-        }
+    unsigned int trmask[BM_MASK_SIZE];
+    int ev_type = 0, *ev_data = 0;
+    long ev_len = 0;
+
+    if (msgLevel(MSG::DEBUG)) debug() << "Waiting for MEP" << endmsg;
+    auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, partitionID);
+    if (sc == MBM_NORMAL)  {
+      slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(ev_data);
+      slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep_header->bytes()};
+      slice.slice_size = static_cast<size_t>(ev_len);
+      auto const* block_header = MEP::block_header(slice.mep_header, 0);
+      slice.packing_factor = block_header->n_banks;
+
+      if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
+    } else if (sc == MBM_REQ_CANCEL) {
+      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      m_buffer_status[i_buffer].writable = true;
+      cancelled = true;
+      if (msgLevel(MSG::DEBUG)) debug() << "Got cancel" << endmsg;
+      if (count_writable() == 0) {
+        m_done = true;
+        break;
       }
     }
 
@@ -1161,7 +1165,7 @@ void MEPProvider::bm_read()
       break;
     }
 
-    assert(cancelled || slice.packing_factor == m_packing_factor);
+    assert(cancelled || slice.packing_factor <= m_packing_factor);
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
-- 
GitLab


From 40550af4486e5e8b1e8c3b23ca68e690870bc07d Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 28 Jul 2021 14:14:30 +0200
Subject: [PATCH 027/120] Add application to test registration with device
 runtime of BM-managed memory

---
 AllenOnline/CMakeLists.txt                    | 17 ++++--
 .../application/test_host_register.cpp        | 60 +++++++++++++++++++
 2 files changed, 72 insertions(+), 5 deletions(-)
 create mode 100644 AllenOnline/application/test_host_register.cpp

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 364a19cc1..7cb09d310 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -8,6 +8,7 @@ gaudi_depends_on_subdirs(Online/OnlineKernel
                          Online/dim
                          Online/RPC
                          Tools/ZeroMQ
+                         Online/OnlineBase
                          Online/Parsers
                          Online/EventBuilding
                          Dumpers/BinaryDumpers)
@@ -47,7 +48,7 @@ gaudi_add_module(AllenOnlineComp
                  src/AllenConfiguration.cpp
                  src/MEPProvider.cpp
                  src/MBMOutput.cpp
-                 INCLUDE_DIRS OnlineBase Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
+                 INCLUDE_DIRS Online/OnlineBase Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
                  LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQLib RPC Parsers EventBuildingLib MDFLib
                                 OnlineKernel ${ALLEN_LIBRARY} BinaryDumpers DAQEventLib
                                 DAQKernelLib GaudiAlgLib PrKernel VPDetLib UTDetLib UTKernelLib AllenOnline
@@ -80,8 +81,14 @@ if (MPI_FOUND)
                        LINK_LIBRARIES AllenOnline MDFLib Boost::program_options MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
   target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
 
-  gaudi_add_executable(test_read_mep
-                       application/read_mep.cpp
-                       INCLUDE_DIRS GaudiKernel
-                       LINK_LIBRARIES AllenOnline MDFLib Boost::program_options)
 endif()
+
+gaudi_add_executable(test_read_mep
+                     application/read_mep.cpp
+                     INCLUDE_DIRS GaudiKernel
+                     LINK_LIBRARIES AllenOnline MDFLib Boost::program_options)
+
+gaudi_add_executable(test_host_register
+                     application/test_host_register.cpp
+                     INCLUDE_DIRS OnlineBase/OnlineBase
+                     LINK_LIBRARIES GaudiKernel OnlineBase ${ALLEN_LIBRARY} ZMQLib CUDA::cudart)
diff --git a/AllenOnline/application/test_host_register.cpp b/AllenOnline/application/test_host_register.cpp
new file mode 100644
index 000000000..a5e3f6714
--- /dev/null
+++ b/AllenOnline/application/test_host_register.cpp
@@ -0,0 +1,60 @@
+#include <Backend/BackendCommon.h>
+#include <MBM/bmdef.h>
+
+#include <iostream>
+#include <thread>
+#include <chrono>
+#include <iomanip>
+
+#define cudaCheck(stmt)                                                                                            \
+  {                                                                     \
+    cudaError_t err = stmt;                                                                                        \
+    if (err != cudaSuccess) {                                                                                      \
+      fprintf(                                                                                                     \
+        stderr, "Failed to run %s\n%s (%d) at %s: %d\n", #stmt, cudaGetErrorString(err), err, __FILE__, __LINE__); \
+      throw std::invalid_argument("cudaCheck failed");                                                             \
+    }                                                                                                              \
+  }
+
+int main() {
+
+  const std::string buffer_name = "Events_0_TDET";
+  auto pn = RTL::processName() + ".0";
+  BMID bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), 32768, BM_COM_FIFO);
+
+  if (bmid == MBM_INV_DESC) {
+    std::cerr << "MBM: Failed to connect to MBM buffer " << buffer_name << std::endl;
+    return -1;
+  }
+
+  // register buffer manager memory with the device runtime
+  size_t buffer_size = 0;
+  char* buffer_address = const_cast<char*>(::mbm_buffer_address(bmid));
+  ::mbm_buffer_size(bmid, &buffer_size);
+
+  std::cout << "MBM buffer: " << reinterpret_cast<void*>(buffer_address) << " " << buffer_size << "\n";
+
+  bool runtime_error = false;
+  try {
+    Allen::set_device(0, 0);
+  } catch (const std::invalid_argument& e) {
+    std::cerr << "Failed to set device: " << e.what() << "\n";
+    runtime_error = true;
+  }
+
+  if (!runtime_error) {
+    try {
+      Allen::host_register(buffer_address, buffer_size, Allen::hostRegisterReadOnly);
+      std::cout << "Successfully registered\n";
+    } catch (const std::invalid_argument& e) {
+      std::cerr << "Failed to register: " << e.what() << "\n";
+    }
+  }
+
+  using namespace std::chrono_literals;
+  std::this_thread::sleep_for(5s);
+
+  ::mbm_exclude(bmid);
+
+  return 0;
+}
-- 
GitLab


From 7165fe6157de75ea3b12e90508c4b653a416887f Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 28 Jul 2021 14:16:43 +0200
Subject: [PATCH 028/120] WIP: fix start-stop logic of MEPProvider BM mode

---
 AllenOnline/src/MEPProvider.cpp | 20 ++++++++++++--------
 1 file changed, 12 insertions(+), 8 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 6c18c8a09..294e67a2c 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -275,15 +275,14 @@ StatusCode MEPProvider::initialize()
   }
   else if (m_source == MEP::ProviderSource::Files) {
     m_read_buffers.resize(n_buffers());
+    // Initialize the current input filename
+    m_current = m_connections.begin();
   }
   else {
     m_bmIDs.resize(n_buffers());
   }
   m_net_slices.resize(n_buffers());
 
-  // Initialize the current input filename
-  m_current = m_connections.begin();
-
   // Allocate space to store event ids
   for (size_t n = 0; n < m_nslices.value(); ++n) {
     m_event_ids[n].reserve(m_events_per_slice);
@@ -341,7 +340,6 @@ StatusCode MEPProvider::stop()
   {
     std::unique_lock<std::mutex> lock {m_control_mutex};
     m_stopping = true;
-    m_started = false;
   }
 
   if (m_source == MEP::ProviderSource::MBM) {
@@ -360,6 +358,11 @@ StatusCode MEPProvider::stop()
     }
   }
 
+  {
+    std::unique_lock<std::mutex> lock {m_control_mutex};
+    m_started = false;
+  }
+
   // Make sure all threads wait for start in case they were waiting
   // for a buffer
   m_mpi_cond.notify_all();
@@ -1100,15 +1103,15 @@ void MEPProvider::bm_read()
     error() << "Number of events makes no sense when receiving from"
             << " the buffer manager: ignoring" << endmsg;
 
-  while (!m_done) {
+  while (!m_done && !m_stopping) {
     // If we've been stopped, wait for start or exit
-    if (!m_started || m_stopping) {
+    if (!m_started) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
       debug() << "Waiting for start" << endmsg;
       m_control_cond.wait(lock, [this] { return m_started || m_done; });
     }
 
-    if (m_done) break;
+    if (m_done || !m_stopping) break;
 
     size_t i_buffer;
     {
@@ -1156,7 +1159,6 @@ void MEPProvider::bm_read()
       if (msgLevel(MSG::DEBUG)) debug() << "Got cancel" << endmsg;
       if (count_writable() == 0) {
         m_done = true;
-        break;
       }
     }
 
@@ -1185,6 +1187,8 @@ void MEPProvider::bm_read()
     else if (!cancelled) {
       debug() << "Prefetch notifying one" << endmsg;
       m_mpi_cond.notify_one();
+    } else if (cancelled) {
+      break;
     }
   }
 }
-- 
GitLab


From ac1abfd962af99ff34329e62ef09896ec6ff7528 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 28 Jul 2021 14:17:53 +0200
Subject: [PATCH 029/120] Re-enable device runtime host registration

---
 AllenOnline/src/MEPProvider.cpp | 22 ++++++++--------------
 1 file changed, 8 insertions(+), 14 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 294e67a2c..db0726a51 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -536,23 +536,17 @@ int MEPProvider::init_bm()
       bmid = ::mbm_connect(m_bmIDs[0], pn.c_str(), partitionID);
     }
 
-    if (bmid == MBM_INV_DESC)   {
+    if (bmid == MBM_INV_DESC) {
       error() << "MBM: Failed to connect to MBM buffer " << buffer_name << endmsg;
       return MBM_ERROR;
     }
-    // else if (b == 0) {
-    //   // register buffer manager memory with the device runtime
-    //   char const* mem = nullptr;
-    //   size_t mem_size = 0;
-    //   auto sc = mbm_buffer_memory(bmid, &mem, &mem_size);
-    //   if (sc == MBM_NORMAL) {
-    //     info() << "MBM mem size: " << mem_size << endmsg;
-    //     // Allen::host_register(const_cast<char*>(mem), mem_size, Allen::hostRegisterDefault);
-    //   } else {
-    //     error() << "MBM: Failed to obtain buffer start and size, MBM" << endmsg;
-    //     return MBM_ERROR;
-    //   }
-    // }
+    else if (b == 0) {
+      // register buffer manager memory with the device runtime
+      size_t buffer_size = 0;
+      char const* buffer_address = mbm_buffer_address(bmid);
+      mbm_buffer_size(bmid, &buffer_size);
+      Allen::host_register(const_cast<char*>(buffer_address), buffer_size, Allen::hostRegisterReadOnly);
+    }
     using namespace std::chrono_literals;
     std::this_thread::sleep_for(50ms);
 
-- 
GitLab


From d125cf2092bc202e1577455dd6271cacda0298d2 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 28 Jul 2021 14:19:09 +0200
Subject: [PATCH 030/120] Fix logging and log format in AllenOnline

---
 AllenOnline/options/Allen.py | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 6c6e632a1..2c44cbf51 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -51,15 +51,16 @@ online_conf.autoStart = False
 online_conf.monitorType = 'MonitorSvc'
 online_conf.logDeviceType = 'RTL::Logger::LogDevice'
 online_conf.logDeviceFormat = '%TIME%LEVEL%-8NODE: %-32PROCESS %-20SOURCE'
-online_conf.OutputLevel = 1
+online_conf.OutputLevel = 3
 online_conf.IOOutputLevel = output_level
 
 if run_online:
     import _fifo_log
+    online_conf.logDeviceType = 'fifo'
+    online_conf.logDeviceFormat = '%-8LEVEL %-24SOURCE'
     _fifo_log.logger_set_tag(partition)
     _fifo_log.logger_start()
 
-
 allen_conf = AllenConfiguration()
 allen_conf.StopTimeout = 5.
 allen_conf.NThreads = 1
@@ -93,7 +94,7 @@ mep_provider.SplitByRun = False
 
 if run_online:
     mep_provider.Source = "MBM"
-    mep_provider.Connections = ["Input"]
+    mep_provider.Connections = ["Events_0"]
     mep_provider.Requests = [
         'EvType=1;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
     ]
@@ -109,7 +110,6 @@ else:
                                        if mep_file.endswith('.mep')])
     mep_provider.LoopOnMEPs = False
 
-
 monSvc = MonitorSvc('MonitorSvc')
 monSvc.ExpandNameInfix = '<proc>'
 monSvc.ExpandCounterServices = True
@@ -124,7 +124,7 @@ app = ApplicationMgr()
 app.MessageSvcType = 'MessageSvc'
 app.OutputLevel = output_level
 messageSvc = MessageSvc('MessageSvc')
-messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
+messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M\n'
 messageSvc.OutputLevel = output_level
 
 
-- 
GitLab


From 18aceaa201db09d30453222b95e45f5eb7e65280 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 29 Jul 2021 17:55:13 +0200
Subject: [PATCH 031/120] Allow MEPProvider to connect to multiple BM buffers.

---
 AllenOnline/AllenOnline/MEPProvider.h |   5 +-
 AllenOnline/src/MEPProvider.cpp       | 111 +++++++++++++++++---------
 2 files changed, 77 insertions(+), 39 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 519449ec1..d2efc1f2c 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -168,7 +168,7 @@ private:
   void mpi_read();
 
   // buffer manager reader thread
-  void bm_read();
+  void bm_read(const std::string& buffer_name);
 
   /**
    * @brief      Function to run in each thread transposing events
@@ -205,9 +205,8 @@ private:
   #endif
 
   std::vector<BufferStatus> m_buffer_status;
-  std::vector<BufferStatus>::iterator m_buffer_transpose;
   std::vector<BufferStatus>::iterator m_buffer_reading;
-  std::optional<std::thread> m_input_thread;
+  std::vector<std::thread> m_input_threads;
 
   // Atomics to flag errors and completion
   std::atomic<bool> m_done = false;
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index db0726a51..180539227 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -235,6 +235,9 @@ StatusCode MEPProvider::initialize()
   m_event_ids.resize(m_nslices.value());
 
   m_buffer_status.resize(n_buffers());
+  for (size_t i = 0; i <  m_buffer_status.size(); ++i) {
+    m_buffer_status[i].index = i;
+  }
 
   if (m_bank_types.empty()) {
     error() << "No bank types specified" << endmsg;
@@ -278,9 +281,6 @@ StatusCode MEPProvider::initialize()
     // Initialize the current input filename
     m_current = m_connections.begin();
   }
-  else {
-    m_bmIDs.resize(n_buffers());
-  }
   m_net_slices.resize(n_buffers());
 
   // Allocate space to store event ids
@@ -308,22 +308,25 @@ StatusCode MEPProvider::start()
   }
 
   // start MPI receive, MEP reading thread or BM thread
-  if (m_source == MEP::ProviderSource::MPI && !m_input_thread) {
+  if (m_source == MEP::ProviderSource::MPI && m_input_threads.empty()) {
 #ifdef HAVE_MPI
-    m_input_thread = std::thread{&MEPProvider::mpi_read, this};
+    m_input_threads.emplace_back(std::thread{&MEPProvider::mpi_read, this});
 #else
     throw StrException {"MPI requested, but no MPI support built in."};
 #endif
   }
-  else if (m_source == MEP::ProviderSource::Files && !m_input_thread) {
-    m_input_thread = std::thread{&MEPProvider::mep_read, this};
+  else if (m_source == MEP::ProviderSource::Files && m_input_threads.empty()) {
+    m_input_threads.emplace_back(std::thread{&MEPProvider::mep_read, this});
   }
   else if (m_source == MEP::ProviderSource::MBM) {
     auto sc = init_bm();
     if (sc != MBM_NORMAL) {
       return StatusCode::FAILURE;
     }
-    m_input_thread = std::thread{&MEPProvider::bm_read, this};
+    for (auto buffer : m_connections.value()) {
+      debug() << "Starting bm_read thread for " << buffer << endmsg;
+      m_input_threads.emplace_back(std::thread{&MEPProvider::bm_read, this, buffer});
+    }
   }
 
   std::unique_lock<std::mutex> lock {m_control_mutex};
@@ -331,7 +334,7 @@ StatusCode MEPProvider::start()
 
   m_started = true;
   m_stopping = false;
-  m_control_cond.notify_one();
+  m_control_cond.notify_all();
   return StatusCode::SUCCESS;
 };
 
@@ -348,13 +351,30 @@ StatusCode MEPProvider::stop()
       if (!m_buffer_status[b].writable) ::mbm_cancel_request(m_bmIDs[b]);
     }
 
-    m_input_thread->join();
-    m_input_thread.reset();
+    for (auto& input_thread : m_input_threads) {
+      input_thread.join();
+    }
+    m_input_threads.clear();
 
-    for (auto bmid : m_bmIDs) {
-      if (bmid != MBM_INV_DESC) {
-        ::mbm_exclude(bmid);
+    for (size_t b = 0; b < m_bmIDs.size(); ++b) {
+      auto bmid = m_bmIDs[b];
+      if (bmid == MBM_INV_DESC) {
+        continue;
+      }
+      else if (b < m_connections.size()) {
+        // register buffer manager memory with the device runtime
+        char const* buffer_address = mbm_buffer_address(bmid);
+        try {
+          Allen::host_unregister(const_cast<char*>(buffer_address));
+          debug() << "Successfully uregistered BM memory for buffer " << b
+                  << " with device runtime." << endmsg;
+        } catch (const std::invalid_argument& e) {
+          error() << "Failed to uregister BM memory for buffer " << b
+                  << " with device runtime : " << e.what() << endmsg;
+          return StatusCode::FAILURE;
+        }
       }
+      ::mbm_exclude(bmid);
     }
   }
 
@@ -517,36 +537,51 @@ StatusCode MEPProvider::init_mpi()
 
 int MEPProvider::init_bm()
 {
-  auto buffer_name = m_connections.value().back();
+  m_bmIDs.resize(n_buffers());
+
   auto const partition = m_allenConfig->partition.value();
   auto const partitionID = m_allenConfig->partitionID.value();
-  if ( m_allenConfig->partitionBuffers.value() ) {
-    std::stringstream stream;
-    stream << std::hex << partitionID;
-    buffer_name += "_";
-    buffer_name += partition.empty() ? stream.str() : partition;
-  }
+
+  std::vector<BMID> first(m_connections.size(), MBM_INV_DESC);
 
   for (size_t b = 0; b < n_buffers(); ++b) {
+    auto buffer_name = m_connections[b % m_connections.size()];
+    if ( m_allenConfig->partitionBuffers.value() ) {
+      std::stringstream stream;
+      stream << std::hex << partitionID;
+      buffer_name += "_";
+      buffer_name += partition.empty() ? stream.str() : partition;
+    }
+
     auto pn = RTL::processName() + "." + std::to_string(b);
     BMID bmid;
-    if (b == 0) {
+    if (b < m_connections.size()) {
       bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), partitionID, BM_COM_FIFO);
-    } else {
-      bmid = ::mbm_connect(m_bmIDs[0], pn.c_str(), partitionID);
-    }
+      if (bmid == MBM_INV_DESC) {
+        error() << "MBM: Failed to connect to MBM buffer " << buffer_name << endmsg;
+        return MBM_ERROR;
+      }
+
+      // Keep track of the master connection BMID
+      first[b % m_connections.size()] = bmid;
 
-    if (bmid == MBM_INV_DESC) {
-      error() << "MBM: Failed to connect to MBM buffer " << buffer_name << endmsg;
-      return MBM_ERROR;
-    }
-    else if (b == 0) {
       // register buffer manager memory with the device runtime
       size_t buffer_size = 0;
       char const* buffer_address = mbm_buffer_address(bmid);
       mbm_buffer_size(bmid, &buffer_size);
-      Allen::host_register(const_cast<char*>(buffer_address), buffer_size, Allen::hostRegisterReadOnly);
+      try {
+        Allen::host_register(const_cast<char*>(buffer_address), buffer_size, Allen::hostRegisterReadOnly);
+        debug() << "Successfully registered BM memory for buffer " << buffer_name
+                << " with device runtime." << endmsg;
+      } catch (const std::invalid_argument& e) {
+        error() << "Failed to register BM memory for buffer " << buffer_name
+                << " with device runtime : " << e.what() << endmsg;
+        return MBM_ERROR;
+      }
+    } else {
+      bmid = ::mbm_connect(first[b % m_connections.size()], pn.c_str(), partitionID);
     }
+
     using namespace std::chrono_literals;
     std::this_thread::sleep_for(50ms);
 
@@ -559,8 +594,8 @@ int MEPProvider::init_bm()
         return MBM_ERROR;
       }
     }
+    info() << "MEP Buffer " << b << " is included in MBM buffer " << buffer_name << endmsg;
     m_bmIDs[b] = bmid;
-    info() << "Buffer " << b << " " << m_bmIDs[b] << endmsg;
   }
   return MBM_NORMAL;
 }
@@ -1089,7 +1124,7 @@ void MEPProvider::mpi_read()
 
 
 // buffer manager reader thread
-void MEPProvider::bm_read()
+void MEPProvider::bm_read(const std::string& buffer_name)
 {
   auto const partitionID = m_allenConfig->partitionID.value();
   auto to_read = this->n_events();
@@ -1097,21 +1132,25 @@ void MEPProvider::bm_read()
     error() << "Number of events makes no sense when receiving from"
             << " the buffer manager: ignoring" << endmsg;
 
+  auto it = std::find(m_connections.begin(), m_connections.end(), buffer_name);
+  size_t const buffer_index = std::distance(m_connections.begin(), it);
+
   while (!m_done && !m_stopping) {
     // If we've been stopped, wait for start or exit
     if (!m_started) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
-      debug() << "Waiting for start" << endmsg;
+      debug() << "bm_read " << buffer_name << " Waiting for start" << endmsg;
       m_control_cond.wait(lock, [this] { return m_started || m_done; });
     }
 
-    if (m_done || !m_stopping) break;
+    if (m_done || m_stopping) break;
 
     size_t i_buffer;
     {
       std::unique_lock<std::mutex> lock {m_mpi_mutex};
       std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+        get_mep_buffer([buffer_index, n_con = m_connections.size()](BufferStatus const& s) {
+          return s.writable && ((s.index % n_con) == buffer_index); }, m_buffer_reading, lock);
       if (m_buffer_reading != m_buffer_status.end()) {
         m_buffer_reading->writable = false;
         assert(m_buffer_reading->work_counter == 0);
-- 
GitLab


From 5411f00a50f4c5b04ab4b08e13b6720b941db140 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 29 Jul 2021 17:56:29 +0200
Subject: [PATCH 032/120] Allow re-use of MEPs loaded into read buffers.

---
 AllenOnline/AllenOnline/MEPProvider.h |   3 +
 AllenOnline/src/MEPProvider.cpp       | 102 ++++++++++++++++++++------
 2 files changed, 81 insertions(+), 24 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index d2efc1f2c..5a188387b 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -308,11 +308,14 @@ private:
     }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
   Gaudi::Property<int> m_window_size {this, "MPIWindow", 4};
   Gaudi::Property<bool> m_non_stop {this, "LoopOnMEPs", false};
+  Gaudi::Property<bool> m_preload {this, "Preload", false};
+  Gaudi::Property<long> m_nevents {this, "EvtMax", 0};
   Gaudi::Property<bool> m_transpose_mep {this, "TransposeMEPs", false};
   Gaudi::Property<bool> m_split_by_run {this, "SplitByRun", true};
 
   // Mapping of receiver card to MPI rank to receive from
   Gaudi::Property<std::map<std::string, int>> m_receivers {this, "Receivers", {}};
+  Gaudi::Property<std::vector<int>> m_buffer_numa {this, "BufferNUMA", {}};
   Gaudi::Property<std::vector<std::string>> m_requests {this, "Requests", {}};
 
 };
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 180539227..27e9ebf1a 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -251,8 +251,40 @@ StatusCode MEPProvider::initialize()
     info() << "Providing banks for " << m_lhcb_bank_types.toString() << endmsg;
   }
 
+  #ifdef HAVE_MPI
+  if (!m_buffer_numa.value().empty() && m_buffer_numa.value().size() != n_buffers()) {
+    error() << "Buffer NUMA domains must be specified for all buffers" << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  // Allocate and initialize topology object.
+  hwloc_topology_init(&m_topology);
+
+  // discover everything, in particular I/O devices like
+  // InfiniBand cards
+  #if HWLOC_API_VERSION >= 0x20000
+  hwloc_topology_set_io_types_filter(m_topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT);
+  #else
+  hwloc_topology_set_flags(m_topology, HWLOC_TOPOLOGY_FLAG_WHOLE_SYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES);
+  #endif
+  // Perform the topology detection.
+  hwloc_topology_load(m_topology);
+
+  [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
+  for (auto domain : m_buffer_numa) {
+    if (domain >= n_numa) {
+      error() << "Illegal NUMA domain specified: " << domain << endmsg;
+      return StatusCode::FAILURE;
+    }
+  }
+  #endif
+
+  std::optional<size_t> n_events;
+  if (m_nevents.value() >= 0)
+    n_events = static_cast<size_t>(m_nevents.value());
+
   init_input(m_nslices, m_events_per_slice, m_bank_types,
-             m_transpose_mep.value() ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, {});
+             m_transpose_mep.value() ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, n_events);
 
   if (m_transpose_mep) {
     info() << "Providing events in Allen layout by transposing MEPs" << endmsg;
@@ -261,7 +293,6 @@ StatusCode MEPProvider::initialize()
     info() << "Providing events in MEP layout" << endmsg;;
   }
 
-  m_buffer_transpose = m_buffer_status.begin();
   m_buffer_reading = m_buffer_status.begin();
 
   auto config = service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
@@ -424,19 +455,6 @@ StatusCode MEPProvider::init_mpi()
 
   m_domains.reserve(m_receivers.size());
 
-  // Allocate and initialize topology object.
-  hwloc_topology_init(&m_topology);
-
-  // discover everything, in particular I/O devices like
-  // InfiniBand cards
-  #if HWLOC_API_VERSION >= 0x20000
-  hwloc_topology_set_io_types_filter(m_topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT);
-  #else
-  hwloc_topology_set_flags(m_topology, HWLOC_TOPOLOGY_FLAG_WHOLE_SYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES);
-  #endif
-  // Perform the topology detection.
-  hwloc_topology_load(m_topology);
-
   hwloc_obj_t osdev = nullptr;
 
   if (!m_receivers.empty()) {
@@ -787,11 +805,13 @@ void MEPProvider::mep_read()
 {
   bool receive_done = false;
 
+  size_t preloaded = n_buffers();
+
   auto to_read = this->n_events();
   if (to_read && msgLevel(MSG::DEBUG)) debug() << "Reading " << *to_read << " events" << endmsg;
   auto to_publish = 0;
 
-  while (!receive_done) {
+  while (!receive_done && !m_read_error) {
     // If we've been stopped, wait for start or exit
     if (!m_started || m_stopping) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
@@ -833,10 +853,36 @@ void MEPProvider::mep_read()
     bool success = false, eof = false;
 
     while (!success || eof) {
-      std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
+      if (!m_preload.value() || preloaded > 0) {
+        std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
+        if (!eof && msgLevel(MSG::DEBUG)) debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
+
+        #ifdef HAVE_MPI
+        if (m_preload.value() && !m_buffer_numa.value().empty()) {
+          auto const numa_node = m_buffer_numa[i_buffer];
+          auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
+          auto s = hwloc_set_area_membind(
+            m_topology, read_buffer.data(), read_buffer.capacity(), numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
+          if (s == 0 && msgLevel(MSG::DEBUG)) {
+            debug() << "Bound preloaded MEP read buffer " << i_buffer <<  " memory to node " << numa_node << endmsg;
+          }
+          else if (s != 0) {
+            error() << "Failed to bind preloaded MEP read buffer " << i_buffer <<  " memory to node " << numa_node << " " << strerror(errno) << endmsg;
+            m_read_error = true;
+            break;
+          }
+        }
+        #endif
+      }
+      else {
+        success = true;
+        eof = false;
+        if (msgLevel(MSG::DEBUG)) {
+          debug() << "Using MEP already read into buffer " << i_buffer << endmsg;
+        }
+      }
 
       if (!eof) {
-        if (msgLevel(MSG::DEBUG)) debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
         if (to_read && success) {
           to_publish = std::min(*to_read, size_t {slice.packing_factor});
           *to_read -= to_publish;
@@ -855,18 +901,24 @@ void MEPProvider::mep_read()
         // Try to open the next file, if there is none, prefetching
         // is done.
         if (!m_read_error && msgLevel(MSG::DEBUG)) {
-          debug() << "Prefetch done: eof and no more files" << endmsg;
+          debug() << "Prefetch done" << endmsg;
         }
         receive_done = true;
+        if (m_preload.value() && preloaded > 0) {
+          error() << "Could not read sufficient MEPs for preloading" << endmsg;
+          m_read_error = true;
+        }
         break;
       }
+      else if (m_preload.value() && preloaded > 0) {
+        --preloaded;
+      }
     }
 
-    if (!m_sizes_known && !allocate_storage(i_buffer)) {
+    if (m_read_error || (!m_sizes_known && !allocate_storage(i_buffer))) {
       m_read_error = true;
       break;
     }
-
     assert(slice.packing_factor <= m_packing_factor);
 
     // Notify a transpose thread that a new buffer of events is
@@ -1246,18 +1298,20 @@ void MEPProvider::transpose(int thread_id)
 
   auto has_intervals = [](BufferStatus const& s) { return !s.intervals.empty(); };
 
+  std::vector<BufferStatus>::iterator buffer_transpose;
+
   while (!m_read_error && !m_transpose_done) {
     // Get a buffer to read from
     {
       std::unique_lock<std::mutex> lock {m_mpi_mutex};
-      std::tie(m_buffer_transpose, i_buffer) = get_mep_buffer(has_intervals, m_buffer_transpose, lock);
+      std::tie(buffer_transpose, i_buffer) = get_mep_buffer(has_intervals, buffer_transpose, lock);
       if (m_transpose_done) {
         break;
       }
-      else if (m_buffer_transpose == m_buffer_status.end()) {
+      else if (buffer_transpose == m_buffer_status.end()) {
         continue;
       }
-      auto& status = *m_buffer_transpose;
+      auto& status = *buffer_transpose;
       assert(!status.intervals.empty());
 
       interval = status.intervals.back();
-- 
GitLab


From ce140935f6b447ea87404d8bd3f4f05ed8c11cc0 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 29 Jul 2021 17:57:14 +0200
Subject: [PATCH 033/120] Fix MBM output trigger mask.

---
 AllenOnline/src/MBMOutput.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index dd7af51f1..cfc3c6469 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -66,7 +66,7 @@ bool MBMOutput::write_buffer(size_t)
     header->setChecksum(c);
   }
 
-  unsigned int mask[] = { ~0x0U, ~0x0U, ~0x0U, m_partitionID };
+  unsigned int mask[] = { m_partitionID, ~0x0U, ~0x0U, ~0x0U };
   void* free_address = nullptr;
   long free_len = 0;
 
-- 
GitLab


From 06d81d62011a35be846e97d90c637781105d64e6 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 29 Jul 2021 17:58:06 +0200
Subject: [PATCH 034/120] Add reminder to TransposeMEP.cpp to switch to using
 source ID (top 5 bits) to identify MFPs instead of bank type

---
 AllenOnline/src/TransposeMEP.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index fab35e3c4..ca67e088e 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -185,6 +185,8 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets(
   uint run_number = 0;
   for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
     auto const& [block_header, block_data] = blocks[i_block];
+    // FIXME: BankType could be an error bank for that
+    // subdetector. Use something better.
     auto lhcb_type = block_header->bank_types()[0];
     auto allen_type = bank_ids[lhcb_type];
     auto n_blocks = banks_count[lhcb_type];
-- 
GitLab


From 24326ebcc3a86eca9e5127e3d76dda3f4647a338 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 29 Jul 2021 17:58:36 +0200
Subject: [PATCH 035/120] Update to Allen.py for latest tests

---
 AllenOnline/options/Allen.py | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 2c44cbf51..6e873158b 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -37,7 +37,7 @@ CondDB().Upgrade = True
 # Get the number of lines from the configuration. This should probably
 # be done internally to Allen and retrieved somehow
 n_lines = 0
-json_file = "/daqarea1/fest/allen/configuration/config.json"
+json_file = "/daqarea1/fest/allen/configuration/passthrough.json"
 with open(json_file) as jf:
     json_config = json.load(jf)
     gs = json_config.get('gather_selections', {})
@@ -88,13 +88,13 @@ mep_provider.NSlices = 8
 mep_provider.EventsPerSlice = 1000
 mep_provider.OutputLevel = 2
 # Number of MEP buffers and number of transpose/offset threads
-mep_provider.BufferConfig = (4, 3)
-mep_provider.TransposeMEPs = True
+mep_provider.BufferConfig = (8, 3)
+mep_provider.TransposeMEPs = False
 mep_provider.SplitByRun = False
 
 if run_online:
     mep_provider.Source = "MBM"
-    mep_provider.Connections = ["Events_0"]
+    mep_provider.Connections = ["Events_0", "Events_1"]
     mep_provider.Requests = [
         'EvType=1;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
     ]
-- 
GitLab


From 1f0daca63ad3713b3dca5283e9f2121fe4203402 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 13 Aug 2021 14:04:45 +0200
Subject: [PATCH 036/120] Updates for the PRR

- Allow either a single thread to read from all MBM buffers in
round-robin fashion or use a thread per MBM buffer
- Properly register and unregister MBM buffers with the device runtime
- Allow configuration of the MBM communication method
- Fix some race conditions
- Fix numa binding to migrate memory and check if binding request was
fulfilled
- Allow loaded MEPs to be reused
---
 AllenOnline/AllenOnline/MEPProvider.h |  35 ++-
 AllenOnline/CMakeLists.txt            |   2 +
 AllenOnline/options/Allen.py          |  11 +-
 AllenOnline/src/AllenApplication.cpp  |   7 +
 AllenOnline/src/MEPProvider.cpp       | 426 +++++++++++++++++---------
 5 files changed, 326 insertions(+), 155 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 5a188387b..5f811a3fd 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -51,6 +51,15 @@ namespace MEP {
                   Files,
                   MBM,
                   MPI)
+
+  meta_enum_class(MBMCom, int,
+                  Unknown = 10,
+                  None = 0,
+                  FIFO = 1,
+                  Asio = 2,
+                  Unix = 4,
+                  SHM1 = 8,
+                  SHM2 = 9)
 }
 
 class AllenConfiguration;
@@ -124,6 +133,8 @@ public:
    */
   void slice_free(size_t slice_index) override;
 
+  bool release_buffers() override;
+
   void event_sizes(
     size_t const slice_index,
     gsl::span<unsigned int const> const selected_events,
@@ -133,6 +144,7 @@ public:
 
   StatusCode initialize() override;
 
+
   StatusCode start() override;
 
   StatusCode stop() override;
@@ -156,7 +168,9 @@ private:
 
   std::tuple<std::vector<BufferStatus>::iterator, size_t> get_mep_buffer(
     std::function<bool(BufferStatus const&)> pred,
+    std::function<bool()> wait_pred,
     std::vector<BufferStatus>::iterator start,
+    std::condition_variable& cond,
     std::unique_lock<std::mutex>& lock);
 
   void set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events);
@@ -179,6 +193,8 @@ private:
    */
   void transpose(int thread_id);
 
+  StatusCode numa_membind(char const* mem, size_t size, int const numa_node) const;
+
   // Slices
   size_t m_packing_factor = 0;
   std::vector<std::vector<char>> m_read_buffers;
@@ -190,10 +206,12 @@ private:
   bool m_stopping = false;
   std::mutex m_control_mutex;
   std::condition_variable m_control_cond;
+  std::condition_variable m_start_cond;
 
   // data members for mpi thread
-  std::mutex m_mpi_mutex;
-  std::condition_variable m_mpi_cond;
+  std::mutex m_buffer_mutex;
+  std::condition_variable m_transpose_cond;
+  std::condition_variable m_receive_cond;
 
   #ifdef HAVE_MPI
   std::vector<std::tuple<int, int>> m_domains;
@@ -206,7 +224,12 @@ private:
 
   std::vector<BufferStatus> m_buffer_status;
   std::vector<BufferStatus>::iterator m_buffer_reading;
+
+  std::mutex m_mbm_mutex;
+  std::vector<bool> m_buffer_event;
   std::vector<std::thread> m_input_threads;
+  size_t m_ninput_threads = 0;
+  size_t m_input_started = 0;
 
   // Atomics to flag errors and completion
   std::atomic<bool> m_done = false;
@@ -225,7 +248,7 @@ private:
 
   // Mutex, condition varaible and queue for parallel transposition of slices
   std::mutex m_transpose_mut;
-  std::condition_variable m_transpose_cond;
+  std::condition_variable m_transposed_cond;
   std::deque<std::tuple<size_t, size_t>> m_transposed;
 
   // Keep track of what slices are free
@@ -253,8 +276,8 @@ private:
 
   // MBM variables
   std::vector<BMID> m_bmIDs;
-
   size_t n_buffers() const { return std::get<0>(m_bufferConfig.value()); }
+  std::vector<char const*> m_registered_buffers;
 
   size_t n_receivers() const { return m_receivers.size(); }
 
@@ -287,6 +310,7 @@ private:
   Gaudi::Property<std::vector<std::string>> m_connections {this, "Connections"};
 
   Gaudi::Property<MEP::ProviderSource> m_source {this, "Source", MEP::ProviderSource::MBM};
+  Gaudi::Property<MEP::MBMCom> m_mbm_com {this, "MBMComMethod", MEP::MBMCom::FIFO};
 
   // number of prefetch buffers and transpose threads
   Gaudi::Property<std::pair<int, int>> m_bufferConfig {this, "BufferConfig", {8, 5},
@@ -309,7 +333,7 @@ private:
   Gaudi::Property<int> m_window_size {this, "MPIWindow", 4};
   Gaudi::Property<bool> m_non_stop {this, "LoopOnMEPs", false};
   Gaudi::Property<bool> m_preload {this, "Preload", false};
-  Gaudi::Property<long> m_nevents {this, "EvtMax", 0};
+  Gaudi::Property<long> m_nevents {this, "EvtMax", -1};
   Gaudi::Property<bool> m_transpose_mep {this, "TransposeMEPs", false};
   Gaudi::Property<bool> m_split_by_run {this, "SplitByRun", true};
 
@@ -317,5 +341,6 @@ private:
   Gaudi::Property<std::map<std::string, int>> m_receivers {this, "Receivers", {}};
   Gaudi::Property<std::vector<int>> m_buffer_numa {this, "BufferNUMA", {}};
   Gaudi::Property<std::vector<std::string>> m_requests {this, "Requests", {}};
+  Gaudi::Property<bool> m_thread_per_buffer {this, "ThreadPerBuffer", true};
 
 };
diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 7cb09d310..80015b9f4 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -81,6 +81,8 @@ if (MPI_FOUND)
                        LINK_LIBRARIES AllenOnline MDFLib Boost::program_options MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
   target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
 
+else()
+  message(STATUS "Could not find MPI")
 endif()
 
 gaudi_add_executable(test_read_mep
diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 6e873158b..1b564e29c 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -63,7 +63,7 @@ if run_online:
 
 allen_conf = AllenConfiguration()
 allen_conf.StopTimeout = 5.
-allen_conf.NThreads = 1
+allen_conf.NThreads = 6
 # Device is a string so the PCI ID can also be given
 # allen_conf.Device = "01:00.0"
 allen_conf.Device = "0"
@@ -82,18 +82,21 @@ elif integration_test:
     allen_conf.Output = "tcp://192.168.1.101:35000"
 else:
     allen_conf.Output = "allen_output.mdf"
+allen_conf.Output = ""
 
 mep_provider = MEPProvider()
 mep_provider.NSlices = 8
-mep_provider.EventsPerSlice = 1000
-mep_provider.OutputLevel = 2
+mep_provider.EventsPerSlice = 10000
+mep_provider.OutputLevel = 3
 # Number of MEP buffers and number of transpose/offset threads
-mep_provider.BufferConfig = (8, 3)
+mep_provider.BufferConfig = (4, 4)
 mep_provider.TransposeMEPs = False
 mep_provider.SplitByRun = False
+mep_provider.ThreadPerBuffer = True
 
 if run_online:
     mep_provider.Source = "MBM"
+    mep_provider.MBMCom = "FIFO"
     mep_provider.Connections = ["Events_0", "Events_1"]
     mep_provider.Requests = [
         'EvType=1;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index d37dd29cf..6106c6e6d 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -104,6 +104,8 @@ int AllenApplication::stop()
       return Online::ONLINE_ERROR;
     }
   }
+
+  m_provider->release_buffers();
   return sc;
 }
 
@@ -326,9 +328,14 @@ std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
   }
 
   auto output = m_allenConfig->output.value();
+  if (output.empty()) {
+    return {};
+  }
+
   std::string connection;
   auto p = output.find("://");
   std::string output_type;
+
   if (p == std::string::npos) {
     output_type = "file";
     connection = output;
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 27e9ebf1a..a119134be 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -64,7 +64,7 @@ EventIDs MEPProvider::event_ids(size_t slice_index, std::optional<size_t> first,
 
 BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) const
 {
-  auto ib = to_integral<BankTypes>(bank_type);
+  auto ib = to_integral(bank_type);
   assert(static_cast<size_t>(ib) < m_slices.size());
   assert(slice_index < m_slices[ib].size());
   auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
@@ -75,9 +75,16 @@ BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) cons
   for (auto s : banks) {
     spans.emplace_back(s);
   }
+
   std::get<1>(bno) = m_transpose_mep.value() ? offsets[offsets_size - 1] : data_size;
   std::get<2>(bno) = offsets;
   std::get<3>(bno) = m_banks_version[ib];
+
+  if (msgLevel(MSG::VERBOSE)) {
+    verbose() << "slice " << std::setw(3) << slice_index << " bank type " << ib
+              << " size " << std::setw(12) << std::get<1>(bno) << endmsg;
+  }
+
   return bno;
 }
 
@@ -100,10 +107,10 @@ MEPProvider::get_slice(std::optional<unsigned int> timeout)
           (m_stopping && n_writable == m_buffer_status.size()));
       };
       if (timeout) {
-        timed_out = !m_transpose_cond.wait_for(lock, std::chrono::milliseconds {*timeout}, wakeup);
+        timed_out = !m_transposed_cond.wait_for(lock, std::chrono::milliseconds {*timeout}, wakeup);
       }
       else {
-        m_transpose_cond.wait(lock, wakeup);
+        m_transposed_cond.wait(lock, wakeup);
       }
     }
     if (!m_read_error && !m_transposed.empty() && (!timeout || (timeout && !timed_out))) {
@@ -143,6 +150,8 @@ void MEPProvider::slice_free(size_t slice_index)
       m_slice_free[slice_index] = true;
       freed = true;
 
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+
       // Clear relation between slice and buffer
       i_buffer = std::get<0>(m_slice_to_buffer[slice_index]);
       auto& status = m_buffer_status[i_buffer];
@@ -158,7 +167,21 @@ void MEPProvider::slice_free(size_t slice_index)
         status.writable = true;
         set_writable = true;
         if (m_source == MEP::ProviderSource::MBM) {
-          ::mbm_free_event(m_bmIDs[i_buffer]);
+          std::unique_lock<std::mutex> lock {m_mbm_mutex};
+          if (m_buffer_event[i_buffer]) {
+            ::mbm_free_event(m_bmIDs[i_buffer]);
+            m_buffer_event[i_buffer] = false;
+          }
+        }
+        if (msgLevel(MSG::VERBOSE)) {
+          verbose() << "Freed MEP buffer " << i_buffer << endmsg;
+          for (auto const& status :  m_buffer_status) {
+            verbose() << std::setw(4) << status.index << std::setw(3) << status.writable << std::setw(4) << status.work_counter;
+            for (auto interval : status.intervals) {
+              verbose() << std::setw(5) << std::get<0>(interval) << std::setw(5) << std::get<1>(interval);
+            }
+            verbose() << endmsg;
+          }
         }
       }
     }
@@ -169,7 +192,7 @@ void MEPProvider::slice_free(size_t slice_index)
   }
   if (set_writable) {
     if (msgLevel(MSG::DEBUG)) debug() << "Set buffer " << i_buffer << " writable" << endmsg;
-    m_mpi_cond.notify_one();
+    m_receive_cond.notify_all();
   }
 }
 
@@ -270,7 +293,7 @@ StatusCode MEPProvider::initialize()
   // Perform the topology detection.
   hwloc_topology_load(m_topology);
 
-  [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
+  auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
   for (auto domain : m_buffer_numa) {
     if (domain >= n_numa) {
       error() << "Illegal NUMA domain specified: " << domain << endmsg;
@@ -331,22 +354,20 @@ StatusCode MEPProvider::start()
     return StatusCode::SUCCESS;
   }
 
-  // Start the transpose threads
-  if (m_transpose_threads.empty() && !m_read_error) {
-    for (int i = 0; i < std::get<1>(m_bufferConfig.value()); ++i) {
-      m_transpose_threads.emplace_back([this, i] { transpose(i); });
-    }
-  }
+  std::unique_lock<std::mutex> lock {m_control_mutex};
+  m_stopping = false;
 
   // start MPI receive, MEP reading thread or BM thread
   if (m_source == MEP::ProviderSource::MPI && m_input_threads.empty()) {
 #ifdef HAVE_MPI
+    m_ninput_threads = 1;
     m_input_threads.emplace_back(std::thread{&MEPProvider::mpi_read, this});
 #else
     throw StrException {"MPI requested, but no MPI support built in."};
 #endif
   }
   else if (m_source == MEP::ProviderSource::Files && m_input_threads.empty()) {
+    m_ninput_threads = 1;
     m_input_threads.emplace_back(std::thread{&MEPProvider::mep_read, this});
   }
   else if (m_source == MEP::ProviderSource::MBM) {
@@ -354,18 +375,35 @@ StatusCode MEPProvider::start()
     if (sc != MBM_NORMAL) {
       return StatusCode::FAILURE;
     }
-    for (auto buffer : m_connections.value()) {
-      debug() << "Starting bm_read thread for " << buffer << endmsg;
-      m_input_threads.emplace_back(std::thread{&MEPProvider::bm_read, this, buffer});
+    m_ninput_threads = m_thread_per_buffer.value() ? m_connections.size() : 1;
+    if (m_thread_per_buffer.value()) {
+      for (auto buffer : m_connections.value()) {
+        debug() << "Starting bm_read thread for " << buffer << endmsg;
+        m_input_threads.emplace_back(std::thread{&MEPProvider::bm_read, this, buffer});
+      }
+    } else {
+      m_input_threads.emplace_back(std::thread{&MEPProvider::bm_read, this, ""});
     }
   }
 
-  std::unique_lock<std::mutex> lock {m_control_mutex};
-  debug() << "Starting" << endmsg;
+  // Start the transpose threads
+  if (m_transpose_threads.empty() && !m_read_error) {
+    for (int i = 0; i < std::get<1>(m_bufferConfig.value()); ++i) {
+      m_transpose_threads.emplace_back([this, i] { transpose(i); });
+    }
+  }
+
+  bool const sizes_known = m_sizes_known;
+  m_start_cond.wait(lock, [this] { return m_input_started == m_input_threads.size(); });
+
+  debug() << "Input threads started" << endmsg;
 
   m_started = true;
-  m_stopping = false;
-  m_control_cond.notify_all();
+  if (!sizes_known) {
+    m_control_cond.notify_one();
+  } else {
+    m_control_cond.notify_all();
+  }
   return StatusCode::SUCCESS;
 };
 
@@ -386,41 +424,46 @@ StatusCode MEPProvider::stop()
       input_thread.join();
     }
     m_input_threads.clear();
-
-    for (size_t b = 0; b < m_bmIDs.size(); ++b) {
-      auto bmid = m_bmIDs[b];
-      if (bmid == MBM_INV_DESC) {
-        continue;
-      }
-      else if (b < m_connections.size()) {
-        // register buffer manager memory with the device runtime
-        char const* buffer_address = mbm_buffer_address(bmid);
-        try {
-          Allen::host_unregister(const_cast<char*>(buffer_address));
-          debug() << "Successfully uregistered BM memory for buffer " << b
-                  << " with device runtime." << endmsg;
-        } catch (const std::invalid_argument& e) {
-          error() << "Failed to uregister BM memory for buffer " << b
-                  << " with device runtime : " << e.what() << endmsg;
-          return StatusCode::FAILURE;
-        }
-      }
-      ::mbm_exclude(bmid);
-    }
   }
 
   {
     std::unique_lock<std::mutex> lock {m_control_mutex};
     m_started = false;
+    m_input_started = 0;
   }
 
   // Make sure all threads wait for start in case they were waiting
   // for a buffer
-  m_mpi_cond.notify_all();
+  m_receive_cond.notify_all();
 
   return StatusCode::SUCCESS;
 };
 
+bool MEPProvider::release_buffers()
+{
+  for (size_t b = 0; b < m_registered_buffers.size(); ++b) {
+    auto const* buffer_address = m_registered_buffers[b];
+    try {
+      Allen::host_unregister(const_cast<char*>(buffer_address));
+      debug() << "Successfully uregistered BM memory for buffer " << b
+              << " with device runtime." << endmsg;
+    } catch (const std::invalid_argument& e) {
+      error() << "Failed to unregister BM memory for buffer " << b
+              << " with device runtime : " << e.what() << endmsg;
+      return false;
+    }
+  }
+  m_registered_buffers.clear();
+
+  for (size_t b = 0; b < m_bmIDs.size(); ++b) {
+    auto bmid = m_bmIDs[b];
+    if (bmid != MBM_INV_DESC) {
+      ::mbm_exclude(bmid);
+    }
+  }
+  return true;
+}
+
 StatusCode MEPProvider::init_mpi()
 {
   #ifdef HAVE_MPI
@@ -485,12 +528,6 @@ StatusCode MEPProvider::init_mpi()
   [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
   assert(static_cast<size_t>(n_numa) == m_domains.size());
 
-  std::vector<hwloc_obj_t> numa_objs(m_receivers.size());
-  for (size_t receiver = 0; receiver < m_receivers.size(); ++receiver) {
-    int numa_node = std::get<1>(m_domains[receiver]);
-    numa_objs[receiver] = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
-  }
-
   std::vector<size_t> packing_factors(m_receivers.size());
   for (size_t receiver = 0; receiver < m_receivers.size(); ++receiver) {
     auto const receiver_rank = std::get<0>(m_domains[receiver]);
@@ -525,15 +562,9 @@ StatusCode MEPProvider::init_mpi()
     // Only bind explicitly if there are multiple receivers,
     // otherwise assume a memory allocation policy is in effect
     if (m_domains.size() > 1) {
-      auto numa_node = i % m_receivers.size();
-      auto const& numa_obj = numa_objs[numa_node];
-      auto s = hwloc_set_area_membind(
-        m_topology, contents, n_bytes, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
-      if (s != 0) {
-        error() << "Failed to bind memory to node " << numa_obj->os_index << " "
-                << strerror(errno) << endmsg;
-        return StatusCode::FAILURE;
-      }
+      auto numa_node = std::get<1>(m_domains[i % m_receivers.size()]);
+      auto sc = numa_membind(contents, n_bytes, numa_node);
+      if (sc.isFailure()) return sc;
     }
 
     Allen::host_register(contents, n_bytes, Allen::hostRegisterDefault);
@@ -560,6 +591,9 @@ int MEPProvider::init_bm()
   auto const partition = m_allenConfig->partition.value();
   auto const partitionID = m_allenConfig->partitionID.value();
 
+  m_buffer_event.resize(n_buffers());
+  m_buffer_event.assign(n_buffers(), false);
+
   std::vector<BMID> first(m_connections.size(), MBM_INV_DESC);
 
   for (size_t b = 0; b < n_buffers(); ++b) {
@@ -574,7 +608,7 @@ int MEPProvider::init_bm()
     auto pn = RTL::processName() + "." + std::to_string(b);
     BMID bmid;
     if (b < m_connections.size()) {
-      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), partitionID, BM_COM_FIFO);
+      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), partitionID, static_cast<int>(m_mbm_com.value()));
       if (bmid == MBM_INV_DESC) {
         error() << "MBM: Failed to connect to MBM buffer " << buffer_name << endmsg;
         return MBM_ERROR;
@@ -587,14 +621,17 @@ int MEPProvider::init_bm()
       size_t buffer_size = 0;
       char const* buffer_address = mbm_buffer_address(bmid);
       mbm_buffer_size(bmid, &buffer_size);
-      try {
-        Allen::host_register(const_cast<char*>(buffer_address), buffer_size, Allen::hostRegisterReadOnly);
-        debug() << "Successfully registered BM memory for buffer " << buffer_name
-                << " with device runtime." << endmsg;
-      } catch (const std::invalid_argument& e) {
-        error() << "Failed to register BM memory for buffer " << buffer_name
-                << " with device runtime : " << e.what() << endmsg;
-        return MBM_ERROR;
+      if (m_registered_buffers.size() < m_connections.size()) {
+        try {
+          Allen::host_register(const_cast<char*>(buffer_address), buffer_size, Allen::hostRegisterReadOnly);
+          debug() << "Successfully registered BM memory for buffer " << buffer_name
+                  << " with device runtime." << endmsg;
+        } catch (const std::invalid_argument& e) {
+          error() << "Failed to register BM memory for buffer " << buffer_name
+                  << " with device runtime : " << e.what() << endmsg;
+          return MBM_ERROR;
+        }
+        m_registered_buffers.push_back(buffer_address);
       }
     } else {
       bmid = ::mbm_connect(first[b % m_connections.size()], pn.c_str(), partitionID);
@@ -658,6 +695,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
     slice.offsets.resize(n_blocks);
     for (auto& offsets : slice.offsets) {
       offsets.resize(m_packing_factor + 1);
+      Allen::host_register(offsets.data(), offsets.capacity(), Allen::hostRegisterDefault);
     }
   }
 
@@ -745,7 +783,6 @@ bool MEPProvider::open_file() const
     }
     else {
       error() << "Failed to open " << *m_current << " " << strerror(errno) << endmsg;
-      m_read_error = true;
       return false;
     }
     ++m_current;
@@ -755,7 +792,9 @@ bool MEPProvider::open_file() const
 
 std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvider::get_mep_buffer(
   std::function<bool(IInputProvider::BufferStatus const&)> pred,
+  std::function<bool()> wait_pred,
   std::vector<IInputProvider::BufferStatus>::iterator start,
+  std::condition_variable& cond,
   std::unique_lock<std::mutex>& lock)
 {
   // Obtain a prefetch buffer to read into, if none is available,
@@ -772,9 +811,9 @@ std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvi
 
   auto it = find_buffer();
   if (it == m_buffer_status.end() && !m_transpose_done) {
-    m_mpi_cond.wait(lock, [this, &it, &find_buffer] {
+    cond.wait(lock, [this, &it, &find_buffer, wait_pred] {
       it = find_buffer();
-      return it != m_buffer_status.end() || m_transpose_done || m_stopping;
+      return it != m_buffer_status.end() || m_read_error  || wait_pred();
     });
   }
   return {it, distance(m_buffer_status.begin(), it)};
@@ -805,7 +844,13 @@ void MEPProvider::mep_read()
 {
   bool receive_done = false;
 
-  size_t preloaded = n_buffers();
+  size_t preloaded = 0;
+  std::vector<bool> preloaded_buffer(n_buffers(), false);
+
+  auto read_error = [this] {
+    m_read_error = true;
+    m_transpose_cond.notify_all();
+  };
 
   auto to_read = this->n_events();
   if (to_read && msgLevel(MSG::DEBUG)) debug() << "Reading " << *to_read << " events" << endmsg;
@@ -813,9 +858,13 @@ void MEPProvider::mep_read()
 
   while (!receive_done && !m_read_error) {
     // If we've been stopped, wait for start or exit
-    if (!m_started || m_stopping) {
+    if (!m_started) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
-      debug() << "Waiting for start" << endmsg;
+      ++m_input_started;
+      if (m_input_started == m_ninput_threads) {
+        m_start_cond.notify_one();
+      }
+      debug() << "mep_read waiting for start" << endmsg;
       m_control_cond.wait(lock, [this] { return m_started || m_done; });
     }
 
@@ -823,15 +872,16 @@ void MEPProvider::mep_read()
 
     // open the first file
     if (!m_input && !open_file()) {
-      m_read_error = true;
-      m_mpi_cond.notify_one();
+      read_error();
       return;
     }
     size_t i_buffer;
     {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
       std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+        get_mep_buffer([](BufferStatus const& s) { return s.writable; },
+                       [this] { return m_stopping; },
+                       m_buffer_reading, m_receive_cond, lock);
       if (m_buffer_reading != m_buffer_status.end()) {
         m_buffer_reading->writable = false;
         assert(m_buffer_reading->work_counter == 0);
@@ -853,24 +903,33 @@ void MEPProvider::mep_read()
     bool success = false, eof = false;
 
     while (!success || eof) {
-      if (!m_preload.value() || preloaded > 0) {
+      if (!m_preload.value() || (preloaded < n_buffers() && !preloaded_buffer[i_buffer])) {
         std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
         if (!eof && msgLevel(MSG::DEBUG)) debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
 
         #ifdef HAVE_MPI
-        if (m_preload.value() && !m_buffer_numa.value().empty()) {
+        if (success && !eof && m_preload.value() && !m_buffer_numa.value().empty()) {
           auto const numa_node = m_buffer_numa[i_buffer];
-          auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
-          auto s = hwloc_set_area_membind(
-            m_topology, read_buffer.data(), read_buffer.capacity(), numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
-          if (s == 0 && msgLevel(MSG::DEBUG)) {
-            debug() << "Bound preloaded MEP read buffer " << i_buffer <<  " memory to node " << numa_node << endmsg;
+          auto const sc = numa_membind(read_buffer.data(), read_buffer.capacity(), numa_node);
+          if (sc.isSuccess()) {
+            info() << "Bound preloaded MEP read buffer " << i_buffer << " memory to node " << numa_node << endmsg;
           }
-          else if (s != 0) {
+          else {
             error() << "Failed to bind preloaded MEP read buffer " << i_buffer <<  " memory to node " << numa_node << " " << strerror(errno) << endmsg;
-            m_read_error = true;
+            read_error();
             break;
           }
+
+          // Register memory with CUDA
+          try {
+            Allen::host_register(read_buffer.data(), read_buffer.capacity(), Allen::hostRegisterDefault);
+          } catch (std::invalid_argument const&) {
+            read_error();
+            break;
+          }
+
+          preloaded_buffer[i_buffer] = true;
+          ++preloaded;
         }
         #endif
       }
@@ -878,12 +937,18 @@ void MEPProvider::mep_read()
         success = true;
         eof = false;
         if (msgLevel(MSG::DEBUG)) {
-          debug() << "Using MEP already read into buffer " << i_buffer << endmsg;
+          debug() << "Using MEP already read into buffer " << i_buffer << "; preloaded " << preloaded << endmsg;
         }
       }
 
+      if (!success) {
+        // Error encountered
+        read_error();
+        break;
+      }
+
       if (!eof) {
-        if (to_read && success) {
+        if (to_read) {
           to_publish = std::min(*to_read, size_t {slice.packing_factor});
           *to_read -= to_publish;
         }
@@ -892,31 +957,23 @@ void MEPProvider::mep_read()
         }
       }
 
-      if (!success) {
-        // Error encountered
-        m_read_error = true;
-        break;
-      }
-      else if ((to_read && *to_read == 0) || (eof && !open_file())) {
+      if ((to_read && *to_read == 0) || (eof && !open_file())) {
         // Try to open the next file, if there is none, prefetching
         // is done.
         if (!m_read_error && msgLevel(MSG::DEBUG)) {
           debug() << "Prefetch done" << endmsg;
         }
         receive_done = true;
-        if (m_preload.value() && preloaded > 0) {
+        if (m_preload.value() && preloaded < n_buffers()) {
           error() << "Could not read sufficient MEPs for preloading" << endmsg;
-          m_read_error = true;
+          read_error();
         }
         break;
       }
-      else if (m_preload.value() && preloaded > 0) {
-        --preloaded;
-      }
     }
 
     if (m_read_error || (!m_sizes_known && !allocate_storage(i_buffer))) {
-      m_read_error = true;
+      read_error();
       break;
     }
     assert(slice.packing_factor <= m_packing_factor);
@@ -925,7 +982,7 @@ void MEPProvider::mep_read()
     // ready. If prefetching is done, wake up all threads
     if (success) {
       {
-        std::unique_lock<std::mutex> lock {m_mpi_mutex};
+        std::unique_lock<std::mutex> lock {m_buffer_mutex};
 
         auto& status = m_buffer_status[i_buffer];
         assert(status.work_counter == 0);
@@ -941,14 +998,13 @@ void MEPProvider::mep_read()
       if (receive_done) {
         m_done = receive_done;
         if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying all" << endmsg;
-        m_mpi_cond.notify_all();
+        m_transpose_cond.notify_all();
       }
       else if (!eof) {
         if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
-        m_mpi_cond.notify_one();
+        m_transpose_cond.notify_one();
       }
     }
-    m_mpi_cond.notify_one();
   }
 }
 
@@ -956,6 +1012,12 @@ void MEPProvider::mep_read()
 void MEPProvider::mpi_read()
 {
   #ifdef HAVE_MPI
+
+  auto read_error = [this] {
+    m_read_error = true;
+    m_transpose_cond.notify_all();
+  };
+
   std::vector<MPI_Request> requests(m_window_size);
 
   // Iterate over the slices
@@ -973,11 +1035,15 @@ void MEPProvider::mpi_read()
   size_t number_of_meps = std::accumulate(n_meps.begin(), n_meps.end(), 0u);
 
   size_t current_mep = 0;
-  while (!m_done && (m_non_stop.value() || current_mep < number_of_meps)) {
+  while (!m_done && !m_read_error && (m_non_stop.value() || current_mep < number_of_meps)) {
     // If we've been stopped, wait for start or exit
-    if (!m_started || m_stopping) {
+    if (!m_started) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
-      debug() << "Waiting for start" << endmsg;
+      ++m_input_started;
+      if (m_input_started == m_ninput_threads) {
+        m_start_cond.notify_one();
+      }
+      debug() << "mpi_read waiting for start" << endmsg;
       m_control_cond.wait(lock, [this] { return m_started || m_done; });
     }
 
@@ -988,9 +1054,11 @@ void MEPProvider::mpi_read()
     // prefetch buffer
     size_t i_buffer;
     {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
       std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([](BufferStatus const& s) { return s.writable; }, m_buffer_reading, lock);
+        get_mep_buffer([](BufferStatus const& s) { return s.writable; },
+                       [this] { return m_stopping; },
+                       m_buffer_reading, m_receive_cond, lock);
       if (m_buffer_reading != m_buffer_status.end()) {
         m_buffer_reading->writable = false;
         assert(m_buffer_reading->work_counter == 0);
@@ -1034,8 +1102,8 @@ void MEPProvider::mpi_read()
         auto s = hwloc_set_area_membind(
           m_topology, contents, slice.slice_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
         if (s != 0) {
-          m_read_error = true;
-          throw GaudiException{name(), "Failed to bind memory to node "s + std::to_string(numa_node) + " " + strerror(errno), StatusCode::FAILURE};
+          read_error();
+          break;
         }
       }
 
@@ -1043,7 +1111,7 @@ void MEPProvider::mpi_read()
       try {
         Allen::host_register(contents, slice.slice_size, Allen::hostRegisterDefault);
       } catch (std::invalid_argument const&) {
-        m_read_error = true;
+        read_error();
         break;
       }
 
@@ -1108,7 +1176,7 @@ void MEPProvider::mpi_read()
     slice.packing_factor = bh->n_banks;
 
     if (!m_sizes_known && !allocate_storage(i_buffer)) {
-      m_read_error = true;
+      read_error();
       break;
     }
 
@@ -1154,14 +1222,13 @@ void MEPProvider::mpi_read()
     // ready. If prefetching is done, wake up all threads
     if (!mpi_error) {
       {
-        std::unique_lock<std::mutex> lock {m_mpi_mutex};
+        std::unique_lock<std::mutex> lock {m_buffer_mutex};
         set_intervals(m_buffer_status[i_buffer].intervals, size_t {slice.packing_factor});
         assert(m_buffer_status[i_buffer].work_counter == 0);
       }
       if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
-      m_mpi_cond.notify_one();
+      m_transpose_cond.notify_one();
     }
-    m_mpi_cond.notify_one();
 
     current_mep++;
   }
@@ -1169,7 +1236,7 @@ void MEPProvider::mpi_read()
   if (!m_done) {
     m_done = true;
     if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying all" << endmsg;
-    m_mpi_cond.notify_all();
+    m_transpose_cond.notify_all();
   }
   #endif
 }
@@ -1178,34 +1245,56 @@ void MEPProvider::mpi_read()
 // buffer manager reader thread
 void MEPProvider::bm_read(const std::string& buffer_name)
 {
+  auto read_error = [this] {
+    m_read_error = true;
+    m_transpose_cond.notify_all();
+    m_receive_cond.notify_all();
+    m_control_cond.notify_all();
+  };
+
   auto const partitionID = m_allenConfig->partitionID.value();
   auto to_read = this->n_events();
   if (to_read)
     error() << "Number of events makes no sense when receiving from"
             << " the buffer manager: ignoring" << endmsg;
 
-  auto it = std::find(m_connections.begin(), m_connections.end(), buffer_name);
-  size_t const buffer_index = std::distance(m_connections.begin(), it);
+  size_t buffer_index = 0;
+  size_t const n_con = m_connections.size();
+  auto select_buffer = [&buffer_index, n_con](BufferStatus const& s) {
+    return s.writable && ((s.index % n_con) == buffer_index);
+  };
+
+  auto buffer_reading = m_buffer_status.begin();
+  if (m_thread_per_buffer.value()) {
+    auto it = std::find(m_connections.begin(), m_connections.end(), buffer_name);
+    buffer_index = std::distance(m_connections.begin(), it);
+    buffer_reading = m_buffer_status.begin() + buffer_index;
+  }
 
-  while (!m_done && !m_stopping) {
+  while (!m_done && !m_stopping && !m_read_error) {
     // If we've been stopped, wait for start or exit
     if (!m_started) {
       std::unique_lock<std::mutex> lock {m_control_mutex};
+      ++m_input_started;
       debug() << "bm_read " << buffer_name << " Waiting for start" << endmsg;
-      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+      if (m_input_started == m_ninput_threads) {
+        m_start_cond.notify_one();
+      }
+      m_control_cond.wait(lock, [this] { return m_started || m_done || m_stopping; });
     }
 
     if (m_done || m_stopping) break;
 
-    size_t i_buffer;
+    size_t i_buffer = 0;
     {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
-      std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([buffer_index, n_con = m_connections.size()](BufferStatus const& s) {
-          return s.writable && ((s.index % n_con) == buffer_index); }, m_buffer_reading, lock);
-      if (m_buffer_reading != m_buffer_status.end()) {
-        m_buffer_reading->writable = false;
-        assert(m_buffer_reading->work_counter == 0);
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+      std::tie(buffer_reading, i_buffer) =
+        get_mep_buffer(select_buffer,
+          [this] { return m_stopping; },
+          buffer_reading, m_receive_cond, lock);
+      if (buffer_reading != m_buffer_status.end()) {
+        buffer_reading->writable = false;
+        assert(buffer_reading->work_counter == 0);
       }
       else {
         continue;
@@ -1216,7 +1305,11 @@ void MEPProvider::bm_read(const std::string& buffer_name)
     }
 
     if (msgLevel(MSG::DEBUG)) {
-      debug() << "Writing to MEP slice index " << i_buffer << endmsg;
+      debug() << "Buffer " << buffer_name << " " << buffer_index << " writing to MEP slice index " << i_buffer << endmsg;
+    }
+
+    if (!m_thread_per_buffer.value()) {
+      buffer_index = (buffer_index + 1) % n_con;
     }
 
     auto& slice = m_net_slices[i_buffer];
@@ -1227,9 +1320,23 @@ void MEPProvider::bm_read(const std::string& buffer_name)
     int ev_type = 0, *ev_data = 0;
     long ev_len = 0;
 
-    if (msgLevel(MSG::DEBUG)) debug() << "Waiting for MEP" << endmsg;
+    if (msgLevel(MSG::DEBUG)) debug() << "Waiting for MEP " << i_buffer << endmsg;
+
+    #ifndef NDEBUG
+    {
+      std::unique_lock<std::mutex> lock {m_mbm_mutex};
+      assert(!m_buffer_event[i_buffer]);
+    }
+    #endif
     auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, partitionID);
     if (sc == MBM_NORMAL)  {
+      {
+        std::unique_lock<std::mutex> lock {m_mbm_mutex};
+        m_buffer_event[i_buffer] = true;
+      }
+
+      // info() << "Got MEP " << i_buffer << endmsg;
+
       slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(ev_data);
       slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep_header->bytes()};
       slice.slice_size = static_cast<size_t>(ev_len);
@@ -1238,18 +1345,23 @@ void MEPProvider::bm_read(const std::string& buffer_name)
 
       if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
     } else if (sc == MBM_REQ_CANCEL) {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
       m_buffer_status[i_buffer].writable = true;
       cancelled = true;
       if (msgLevel(MSG::DEBUG)) debug() << "Got cancel" << endmsg;
-      if (count_writable() == 0) {
+      if (count_writable() == n_buffers()) {
         m_done = true;
       }
     }
 
-    if (!m_sizes_known && !allocate_storage(i_buffer)) {
-      m_read_error = true;
-      break;
+    if (!m_sizes_known && !cancelled) {
+      if (!allocate_storage(i_buffer)) {
+        read_error();
+        break;
+      }
+      else {
+        m_control_cond.notify_all();
+      }
     }
 
     assert(cancelled || slice.packing_factor <= m_packing_factor);
@@ -1257,7 +1369,7 @@ void MEPProvider::bm_read(const std::string& buffer_name)
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
     if (!cancelled) {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
 
       auto& status = m_buffer_status[i_buffer];
       assert(status.work_counter == 0);
@@ -1267,11 +1379,11 @@ void MEPProvider::bm_read(const std::string& buffer_name)
 
     if (m_done) {
       debug() << "Prefetch notifying all" << endmsg;
-      m_mpi_cond.notify_all();
+      m_transpose_cond.notify_all();
     }
     else if (!cancelled) {
       debug() << "Prefetch notifying one" << endmsg;
-      m_mpi_cond.notify_one();
+      m_transpose_cond.notify_one();
     } else if (cancelled) {
       break;
     }
@@ -1296,16 +1408,21 @@ void MEPProvider::transpose(int thread_id)
   bool good = false, transpose_full = false;
   size_t n_transposed = 0;
 
-  auto has_intervals = [](BufferStatus const& s) { return !s.intervals.empty(); };
+  auto has_intervals = [thread_id, n_input = m_ninput_threads](BufferStatus const& s) {
+    return !s.intervals.empty() && ((s.index % n_input) == (thread_id % n_input));
+  };
 
-  std::vector<BufferStatus>::iterator buffer_transpose;
+  std::vector<BufferStatus>::iterator buffer_transpose = m_buffer_status.begin();
 
   while (!m_read_error && !m_transpose_done) {
     // Get a buffer to read from
     {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
-      std::tie(buffer_transpose, i_buffer) = get_mep_buffer(has_intervals, buffer_transpose, lock);
-      if (m_transpose_done) {
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+      std::tie(buffer_transpose, i_buffer) = get_mep_buffer(has_intervals,
+                                                            [this]() -> bool { return m_transpose_done; },
+                                                            buffer_transpose,
+                                                            m_transpose_cond, lock);
+      if (m_transpose_done || m_read_error) {
         break;
       }
       else if (buffer_transpose == m_buffer_status.end()) {
@@ -1394,12 +1511,15 @@ void MEPProvider::transpose(int thread_id)
       std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
         m_slices, *slice_index, m_bank_ids, m_bank_types, m_banks_count, event_ids, slice.blocks, interval);
       if (msgLevel(MSG::DEBUG)) {
-        debug() << "Transpose " << thread_id << ": Calculated MEP offsets for slice " << *slice_index << endmsg;
+        debug() << "Transpose " << thread_id << ": Calculated MEP offsets for slice " << *slice_index
+                << "; good: " << good << "; full: " << transpose_full
+                << "; n_transposed: " << n_transposed << endmsg;
+
       }
     }
 
     if (m_read_error || !good) {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
       auto& status = m_buffer_status[i_buffer];
       --status.work_counter;
       m_read_error = true;
@@ -1412,14 +1532,14 @@ void MEPProvider::transpose(int thread_id)
       std::unique_lock<std::mutex> lock {m_transpose_mut};
       m_transposed.emplace_back(*slice_index, n_transposed);
     }
-    m_transpose_cond.notify_one();
+    m_transposed_cond.notify_one();
     slice_index.reset();
 
     // Check if the read buffer is now empty. If it is, it can be
     // reused, otherwise give it to another transpose thread once a
     // new target slice is available
     {
-      std::unique_lock<std::mutex> lock {m_mpi_mutex};
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
 
       auto& status = m_buffer_status[i_buffer];
       --status.work_counter;
@@ -1436,3 +1556,17 @@ void MEPProvider::transpose(int thread_id)
     }
   }
 }
+
+StatusCode MEPProvider::numa_membind(char const* mem, size_t size, int const numa_node) const {
+  auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
+  auto s = hwloc_set_area_membind(
+    m_topology, mem, size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET | HWLOC_MEMBIND_STRICT | HWLOC_MEMBIND_MIGRATE);
+  if (s != 0) {
+    error() << "Failed to bind memory to node " << numa_obj->os_index << " "
+            << strerror(errno) << endmsg;
+    return StatusCode::FAILURE;
+  }
+  else {
+    return StatusCode::SUCCESS;
+  }
+}
-- 
GitLab


From f2ab605623c8854fc28c4202b2d7bc553333cb02 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 13 Aug 2021 14:09:34 +0200
Subject: [PATCH 037/120] Sort MFPs by (source ID, bank type)

Note lookup of MFPs by bank type should be changed to lookup by source
ID. The 5 most significant bits of the source IDs are
subdetector-specific and can be used to group MFPs.
---
 AllenOnline/src/TransposeMEP.cpp | 32 +++++++++++++++++++++++---------
 1 file changed, 23 insertions(+), 9 deletions(-)

diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index ca67e088e..47c507ae6 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -50,17 +50,31 @@ MEP::fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& mep
   return {true, count, versions};
 }
 
-void MEP::find_blocks(MEP::MEP_header const* mep_header, gsl::span<char const> const& buffer_span, Blocks& blocks)
-{
-  // Fill blocks
-  for (size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
+void MEP::find_blocks( MEP::MEP_header const* mep_header, gsl::span<char const> const& buffer_span, Blocks& blocks ) {
+  // Fill blocks in temporary container
+  Blocks tmp{blocks};
+  for ( size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block ) {
     // block offsets are in 4-byte words with respect to the start of the MEP header
-    auto block_offset = mep_header->offsets()[i_block] * sizeof(uint32_t);
-    char const* block_start = buffer_span.data() + block_offset;
-    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>(block_start);
-    gsl::span<char const> block_data {block_start + block_header->header_size(), block_header->bytes() -  block_header->header_size()};
-    blocks[i_block] = std::tuple {block_header, std::move(block_data)};
+    auto                   block_offset = mep_header->offsets()[i_block] * sizeof( uint32_t );
+    char const*            block_start  = buffer_span.data() + block_offset;
+    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>( block_start );
+    gsl::span<char const>  block_data{block_start + block_header->header_size(),
+                                     block_header->bytes() - block_header->header_size()};
+    tmp[i_block] = std::tuple{block_header, std::move( block_data )};
   }
+
+  auto const*         src_ids = mep_header->src_ids();
+  std::vector<size_t> perm( tmp.size() );
+  std::iota( perm.begin(), perm.end(), 0U );
+  std::sort( perm.begin(), perm.end(), [&tmp, src_ids]( size_t a, size_t b ) {
+    if ( src_ids[a] == src_ids[b] ) {
+      return std::get<0>( tmp[a] )->bank_types()[0] < std::get<0>( tmp[b] )->bank_types()[0];
+    } else {
+      return src_ids[a] < src_ids[b];
+    }
+  } );
+
+  for ( size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block ) { blocks[i_block] = tmp[perm[i_block]]; }
 }
 
 void MEP::fragment_offsets(MEP::Blocks const& blocks, MEP::SourceOffsets& offsets)
-- 
GitLab


From e516a7622c6cd135ec3295251603722c6f11d510 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 13 Aug 2021 14:12:23 +0200
Subject: [PATCH 038/120] Allow reading of MEP files larger than 2 GB

---
 AllenOnline/application/read_mep.cpp |  17 +-
 AllenOnline/src/ReadMEP.cpp          |  29 ++-
 AllenOnline/src/TransposeMEP.cpp     | 313 ++++++++++++---------------
 3 files changed, 160 insertions(+), 199 deletions(-)

diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index f395bf1fa..63ac75f40 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -67,24 +67,25 @@ int main(int argc, char* argv[])
   for (size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep) {
 
     std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(input, data);
+    auto const* mep_header = slice.mep_header;
     if (!success) {
       return 1;
     }
     else {
-      cout << "Read mep with packing factor " << slice.packing_factor << "\n";
+      cout << "Read mep with packing factor " << slice.packing_factor << " #MFPs: " << mep_header->n_MFPs << "\n";
     }
 
     if (i_mep == 0) {
       event_ids.reserve(slice.packing_factor);
-      slice.blocks.resize(slice.mep_header->n_MFPs, MEP::Blocks::value_type{});
-      slice.offsets.resize(slice.mep_header->n_MFPs);
+      slice.blocks.resize(mep_header->n_MFPs, MEP::Blocks::value_type{});
+      slice.offsets.resize(mep_header->n_MFPs);
       for (auto& offsets : slice.offsets) {
         offsets.resize(slice.packing_factor + 1);
       }
     }
 
-    MEP::find_blocks(slice.mep_header, slice.mep_data, slice.blocks);
-    auto [sucess, banks_count, banks_version] = MEP::fill_counts(slice.mep_header, slice.mep_data, bank_ids);
+    MEP::find_blocks(mep_header, slice.mep_data, slice.blocks);
+    auto [sucess, banks_count, banks_version] = MEP::fill_counts(mep_header, slice.mep_data, bank_ids);
 
     if (i_mep == 0) {
       bank_slices = allocate_slices(1, bank_types,
@@ -99,12 +100,12 @@ int main(int argc, char* argv[])
       reset_slice(bank_slices, 0, bank_types, event_ids, true);
     }
 
-
     for (auto const& [block_header, block_data] : slice.blocks) {
       // block offsets are in number of 4-byte words
       auto lhcb_type = int {block_header->bank_types()[0]};
 
       cout << "fragment"
+           << " magic: 0x" << std::hex << block_header->magic << std::dec
            << " packing: " << std::setw(4) << block_header->n_banks << " event_id: " << std::setw(6)
            << block_header->ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
            << (block_header->src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{block_header->block_version}
@@ -119,10 +120,6 @@ int main(int argc, char* argv[])
                      slice.blocks,
                      {0ul, slice.packing_factor},
                      false);
-
-    for (size_t i = 0; i < std::min(10ul, event_ids.size()); ++i) {
-      cout << std::get<0>(event_ids[i]) << " " << std::get<1>(event_ids[i]) << "\n";
-    }
   }
   return 0;
 }
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index 25d7d0466..3ec2371d4 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -10,6 +10,7 @@
 #include <vector>
 #include <cstring>
 #include <cassert>
+#include <limits>
 
 #include <gsl/gsl>
 
@@ -41,12 +42,13 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   buffer.resize(sizeof(MEP::MEP_header));
 
   // Read the first few words of the MEP header
-  auto n_bytes = input.read(&buffer[0], sizeof(MEP::MEP_header));
-  if (n_bytes == 0) {
+  // Why is the number of bytes returned as an int...
+  int read_sc = input.read(&buffer[0], sizeof(MEP::MEP_header));
+  if (read_sc == 0) {
     cout << "Cannot read more data (Header). End-of-File reached.\n";
     return {true, true, nullptr, 0u, {}};
   }
-  else if (n_bytes < 0) {
+  else if (read_sc < 0) {
     cerr << "Failed to read header " << strerror(errno) << "\n";
     return {false, false, nullptr, 0u, {}};
   }
@@ -61,17 +63,24 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   // The size of the MEP has been read, so allocate space for that
   // (don't forget to redo the pointers in case the memory was
   // reallocated elsewhere)
-  auto data_size = static_cast<size_t>(mep_header->bytes());
+  size_t data_size = mep_header->bytes();
   buffer.resize(data_size);
   char* mep_buffer = &buffer[0];
   mep_header = reinterpret_cast<MEP::MEP_header const*>(mep_buffer);
 
-  // Read the rest of the MEP now that the size is known
-  n_bytes = input.read(mep_buffer + sizeof(MEP::MEP_header),
-    static_cast<int>(data_size - sizeof(MEP::MEP_header)));
-  if (n_bytes <= 0) {
-    cerr << "Failed to read MEP" << strerror(errno) << "\n";
-    return {false, false, nullptr, 0u, {}};
+  char* pos = mep_buffer + sizeof(MEP::MEP_header);
+  size_t remaining = data_size - sizeof(MEP::MEP_header);
+  while (remaining > 0) {
+    size_t chunk = std::min(remaining, static_cast<size_t>(std::numeric_limits<int>::max() / 2));
+    // Read the next chunk
+    cout << "Reading " << chunk << " bytes\n";
+    read_sc = input.read(pos, static_cast<int>(chunk));
+    if (read_sc <= 0) {
+      cerr << "Failed to read MEP" << strerror(errno) << "\n";
+      return {false, false, nullptr, 0u, {}};
+    }
+    remaining -= chunk;
+    pos += chunk;
   }
 
   // Get the packing factor
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 47c507ae6..04b718ed4 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -9,42 +9,38 @@
 #include <AllenOnline/TransposeMEP.h>
 
 namespace {
-  std::tuple<unsigned, unsigned long> decode_odin(unsigned const* odin_data) {
-    unsigned const run_number = odin_data[LHCb::ODIN::Data::RunNumber];
-    unsigned long evt_hi = odin_data[LHCb::ODIN::Data::L0EventIDHi];
-    unsigned long evt_lo = odin_data[LHCb::ODIN::Data::L0EventIDLo];
-    unsigned long event_number = (evt_hi << 32) | evt_lo;
+  std::tuple<unsigned, unsigned long> decode_odin( unsigned const* odin_data ) {
+    unsigned const run_number   = odin_data[LHCb::ODIN::Data::RunNumber];
+    unsigned long  evt_hi       = odin_data[LHCb::ODIN::Data::L0EventIDHi];
+    unsigned long  evt_lo       = odin_data[LHCb::ODIN::Data::L0EventIDLo];
+    unsigned long  event_number = ( evt_hi << 32 ) | evt_lo;
     return {run_number, event_number};
   }
-}
+} // namespace
 
 std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
-MEP::fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& mep_span, std::vector<int> const& bank_ids)
-{
+MEP::fill_counts( MEP::MEP_header const* header, gsl::span<char const> const& mep_span,
+                  std::vector<int> const& bank_ids ) {
   // info_cout << "EB header: "
   //   << header.n_blocks << ", "
   //   << header.packing_factor << ", "
   //   << header.reserved << ", "
   //   << header.mep_size << "\n";
 
-  auto header_size = header->header_size();
-  gsl::span<char const> block_span {mep_span.data() + header_size, mep_span.size() - header_size};
-  std::array<unsigned int, LHCb::NBankTypes> count {0};
-  std::array<int, NBankTypes> versions {0};
-  for (size_t i = 0; i < header->n_MFPs; ++i) {
-    auto const* bh = block_header(header, i);
+  auto                                       header_size = header->header_size();
+  gsl::span<char const>                      block_span{mep_span.data() + header_size, mep_span.size() - header_size};
+  std::array<unsigned int, LHCb::NBankTypes> count{0};
+  std::array<int, NBankTypes>                versions{0};
+  for ( size_t i = 0; i < header->n_MFPs; ++i ) {
+    auto const* bh = block_header( header, i );
 
     // info_cout << "EB BlockHeader: "
     //   << bh.event_id << ", " << bh.n_frag << ", " << bh.reserved << ", " << bh.block_size << "\n";
     auto type = bh->bank_types()[0];
-    if (type < LHCb::RawBank::LastType) {
-      ++count[type];
-    }
+    if ( type < LHCb::RawBank::LastType ) { ++count[type]; }
 
     auto const allen_type = bank_ids[type];
-    if (allen_type != -1) {
-      versions[allen_type] = bh->block_version;
-    }
+    if ( allen_type != -1 ) { versions[allen_type] = bh->block_version; }
   }
 
   return {true, count, versions};
@@ -77,91 +73,82 @@ void MEP::find_blocks( MEP::MEP_header const* mep_header, gsl::span<char const>
   for ( size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block ) { blocks[i_block] = tmp[perm[i_block]]; }
 }
 
-void MEP::fragment_offsets(MEP::Blocks const& blocks, MEP::SourceOffsets& offsets)
-{
+void MEP::fragment_offsets( MEP::Blocks const& blocks, MEP::SourceOffsets& offsets ) {
 
   // Reset input offsets
-  for (auto& o : offsets) {
-    std::fill(o.begin(), o.end(), 0);
-  }
+  for ( auto& o : offsets ) { std::fill( o.begin(), o.end(), 0 ); }
 
   // Loop over all bank sizes in all blocks
-  for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
+  for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
     auto const& [block_header, block_data] = blocks[i_block];
-    auto& o = offsets[i_block];
-    uint32_t fragment_offset = 0;
-
-    for (size_t i = 0; i < block_header->n_banks; ++i) {
+    auto&       o                          = offsets[i_block];
+    uint32_t    fragment_offset            = 0;
+    auto const* bank_sizes                 = block_header->bank_sizes();
+    for ( size_t i = 0; i < block_header->n_banks; ++i ) {
       o[i] = fragment_offset;
-      fragment_offset += block_header->bank_sizes()[i];
+      fragment_offset += bank_sizes[i];
     }
   }
 }
 
-size_t MEP::allen_offsets(
-  Allen::Slices& slices,
-  int const slice_index,
-  std::vector<int> const& bank_ids,
-  std::unordered_set<BankTypes> const& bank_types,
-  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
-  MEP::Blocks const& blocks,
-  MEP::SourceOffsets const& input_offsets,
-  std::tuple<size_t, size_t> const& interval,
-  bool split_by_run)
-{
+size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
+                           std::unordered_set<BankTypes> const&              bank_types,
+                           std::array<unsigned int, LHCb::NBankTypes> const& banks_count, MEP::Blocks const& blocks,
+                           MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval,
+                           bool split_by_run ) {
 
   auto [event_start, event_end] = interval;
 
   // Loop over all bank sizes in all blocks
-  for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
+  for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
     auto const& [block_header, block_data] = blocks[i_block];
-    auto lhcb_type = block_header->bank_types()[0];
-    auto allen_type = bank_ids[lhcb_type];
-    auto& source_offsets = input_offsets[i_block];
-    uint run_number = 0;
-    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
-      for (size_t i = event_start; i < event_end; ++i) {
+    auto const* bank_sizes                 = block_header->bank_sizes();
+    auto        lhcb_type                  = block_header->bank_types()[0];
+    auto        allen_type                 = bank_ids[lhcb_type];
+    auto&       source_offsets             = input_offsets[i_block];
+    uint        run_number                 = 0;
+    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
+      for ( size_t i = event_start; i < event_end; ++i ) {
         // First check for run changes in ODIN banks
-        if (split_by_run && lhcb_type == LHCb::RawBank::ODIN) {
+        if ( split_by_run && lhcb_type == LHCb::RawBank::ODIN ) {
           // decode ODIN banks to check for run changes
-          auto odin_data = reinterpret_cast<unsigned int const*>(block_data.data() + source_offsets[i]);
+          auto           odin_data = reinterpret_cast<unsigned int const*>( block_data.data() + source_offsets[i] );
           unsigned const odin_run_number = odin_data[LHCb::ODIN::Data::RunNumber];
           // if splitting by run, check all events have same run number
-          if (i == event_start) {
+          if ( i == event_start ) {
             run_number = odin_run_number;
-          }
-          else if (odin_run_number != run_number) {
+          } else if ( odin_run_number != run_number ) {
             event_end = i;
             break;
           }
         }
         // Anticipate offset structure already here, i.e. don't assign to the first one
-        auto idx = i - event_start + 1;
-        auto& event_offsets = std::get<2>(slices[allen_type][slice_index]);
+        auto  idx           = i - event_start + 1;
+        auto& event_offsets = std::get<2>( slices[allen_type][slice_index] );
 
         // Allen raw bank format has the sourceID followed by the raw bank data
-        event_offsets[idx] += sizeof(uint32_t) + block_header->bank_sizes()[i];
+        event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i];
       }
     }
   }
 
   // Prefix sum over sizes per bank type per event to get the output
   // "Allen" offsets per bank type per event
-  size_t n_frag = (event_end - event_start);
-  for (size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type) {
+  size_t n_frag = ( event_end - event_start );
+  for ( size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type ) {
     auto allen_type = bank_ids[lhcb_type];
-    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
+    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
       auto& [slice, slice_size, event_offsets, offsets_size] = slices[allen_type][slice_index];
-      event_offsets[0] = 0;
-      auto preamble_words = 2 + banks_count[lhcb_type];
-      for (size_t i = 1; i <= (event_end - event_start) && i <= n_frag; ++i) {
+      event_offsets[0]                                       = 0;
+      auto preamble_words                                    = 2 + banks_count[lhcb_type];
+      for ( size_t i = 1; i <= ( event_end - event_start ) && i <= n_frag; ++i ) {
 
         // Allen raw bank format has the number of banks and the bank
         // offsets in a preamble
-        event_offsets[i] += preamble_words * sizeof(uint32_t) + event_offsets[i - 1];
+        event_offsets[i] += preamble_words * sizeof( uint32_t ) + event_offsets[i - 1];
 
         // Check for sufficient space
-        if (event_offsets[i] > slice_size) {
+        if ( event_offsets[i] > slice_size ) {
           n_frag = i - 1;
           break;
         }
@@ -170,105 +157,97 @@ size_t MEP::allen_offsets(
   }
 
   // Set offsets_size here to make sure it's consistent with the max
-  for (size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type) {
+  for ( size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type ) {
     auto allen_type = bank_ids[lhcb_type];
-    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
-      auto& offsets_size = std::get<3>(slices[allen_type][slice_index]);
-      offsets_size = n_frag + 1;
+    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
+      auto& offsets_size = std::get<3>( slices[allen_type][slice_index] );
+      offsets_size       = n_frag + 1;
     }
   }
   return n_frag;
 }
 
-std::tuple<bool, bool, size_t> MEP::mep_offsets(
-  Allen::Slices& slices,
-  int const slice_index,
-  std::vector<int> const& bank_ids,
-  std::unordered_set<BankTypes> const& bank_types,
-  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
-  EventIDs& event_ids,
-  MEP::Blocks const& blocks,
-  std::tuple<size_t, size_t> const& interval,
-  bool split_by_run)
-{
+std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int const slice_index,
+                                                 std::vector<int> const&                           bank_ids,
+                                                 std::unordered_set<BankTypes> const&              bank_types,
+                                                 std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+                                                 EventIDs& event_ids, MEP::Blocks const& blocks,
+                                                 std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
 
   auto [event_start, event_end] = interval;
 
-  unsigned char prev_type = 0;
-  size_t offset_index = 0;
-  uint run_number = 0;
-  for (size_t i_block = 0; i_block < blocks.size(); ++i_block) {
+  unsigned char prev_type    = 0;
+  size_t        offset_index = 0;
+  uint          run_number   = 0;
+  for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
     auto const& [block_header, block_data] = blocks[i_block];
     // FIXME: BankType could be an error bank for that
     // subdetector. Use something better.
-    auto lhcb_type = block_header->bank_types()[0];
-    auto allen_type = bank_ids[lhcb_type];
-    auto n_blocks = banks_count[lhcb_type];
+    auto        lhcb_type  = block_header->bank_types()[0];
+    auto        allen_type = bank_ids[lhcb_type];
+    auto        n_blocks   = banks_count[lhcb_type];
+    auto const* bank_sizes = block_header->bank_sizes();
 
     // Decode ODIN banks
-    if (lhcb_type == LHCb::RawBank::ODIN) {
+    if ( lhcb_type == LHCb::RawBank::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
       unsigned fragment_offset = 0;
-      for (unsigned i_event = 0; i_event < event_end; ++i_event) {
-        if (i_event >= event_start) {
-          auto const* odin_data = reinterpret_cast<unsigned int const*>(block_data.data() + fragment_offset);
-          auto const [odin_run_number, event_number] = decode_odin(odin_data);
+      for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
+        if ( i_event >= event_start ) {
+          auto const* odin_data = reinterpret_cast<unsigned int const*>( block_data.data() + fragment_offset );
+          auto const [odin_run_number, event_number] = decode_odin( odin_data );
 
           // if splitting by run, check all events have same run number
-          if (i_event == event_start) {
+          if ( i_event == event_start ) {
             run_number = odin_run_number;
-          }
-          else if (split_by_run && odin_run_number != run_number) {
+          } else if ( split_by_run && odin_run_number != run_number ) {
             event_end = i_event;
             break;
           }
-          event_ids.emplace_back(odin_run_number, event_number);
+          event_ids.emplace_back( odin_run_number, event_number );
         }
-        fragment_offset += block_header->bank_sizes()[i_event];
+        fragment_offset += bank_sizes[i_event];
       }
     }
 
-    if (allen_type != -1 && bank_types.count(BankTypes {allen_type})) {
+    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
       auto& [spans, data_size, event_offsets, offsets_size] = slices[allen_type][slice_index];
+      auto const* bank_sizes                                = block_header->bank_sizes();
 
       // Calculate block offset and size
       size_t interval_offset = 0, interval_size = 0;
-      for (size_t i = 0; i < event_start; ++i) {
-        interval_offset += block_header->bank_sizes()[i];
-      }
-      for (size_t i = event_start; i < event_end; ++i) {
-        interval_size += block_header->bank_sizes()[i];
-      }
+      for ( size_t i = 0; i < event_start; ++i ) { interval_offset += bank_sizes[i]; }
+      for ( size_t i = event_start; i < event_end; ++i ) { interval_size += bank_sizes[i]; }
 
       // Calculate offsets
-      if (lhcb_type != prev_type) {
-        event_offsets[0] = banks_count[lhcb_type];
-        event_offsets[1] = event_end - event_start;
+      if ( lhcb_type != prev_type ) {
+        event_offsets[0]            = banks_count[lhcb_type];
+        event_offsets[1]            = event_end - event_start;
         event_offsets[2 + n_blocks] = 0;
-        offset_index = 0;
-        prev_type = lhcb_type;
+        offset_index                = 0;
+        prev_type                   = lhcb_type;
       }
 
       // Store source ID, mask top 5 bits which are detector ID
       event_offsets[2 + offset_index] = block_header->src_id & 0x7FF;
 
       // Initialize the first offsets using the block sizes,
-      if (offset_index < banks_count[lhcb_type] - 1) {
+      if ( offset_index < banks_count[lhcb_type] - 1 ) {
         event_offsets[2 + n_blocks + offset_index + 1] = event_offsets[2 + n_blocks + offset_index] + interval_size;
       }
 
       // Fill fragment offsets
       size_t oi = 0, idx = 0;
-      for (size_t i = event_start; i < event_end; ++i) {
-        idx = i - event_start + 1;
-        oi = 2 + n_blocks * (1 + idx) + offset_index;
-        event_offsets[oi] = event_offsets[oi - n_blocks] + block_header->bank_sizes()[i];
+      for ( size_t i = event_start; i < event_end; ++i ) {
+        idx               = i - event_start + 1;
+        oi                = 2 + n_blocks * ( 1 + idx ) + offset_index;
+        event_offsets[oi] = event_offsets[oi - n_blocks] + bank_sizes[i];
       }
       // Update offsets_size
       offsets_size = oi;
 
       // Store block span for this interval
-      spans.emplace_back(const_cast<char*>(block_data.data()) + interval_offset, interval_size);
+      spans.emplace_back( const_cast<char*>( block_data.data() ) + interval_offset, interval_size );
       data_size += interval_size;
 
       ++offset_index;
@@ -277,18 +256,11 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets(
   return {true, false, event_end - event_start};
 }
 
-bool MEP::transpose_event(
-  Allen::Slices& slices,
-  int const slice_index,
-  std::vector<int> const& bank_ids,
-  std::unordered_set<BankTypes> const& bank_types,
-  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
-  EventIDs& event_ids,
-  MEP::MEP_header const* mep_header,
-  MEP::Blocks const& blocks,
-  MEP::SourceOffsets const& input_offsets,
-  std::tuple<size_t, size_t> const& interval)
-{
+bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
+                           std::unordered_set<BankTypes> const&              bank_types,
+                           std::array<unsigned int, LHCb::NBankTypes> const& banks_count, EventIDs& event_ids,
+                           MEP::MEP_header const* mep_header, MEP::Blocks const& blocks,
+                           MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval ) {
   auto [start_event, end_event] = interval;
 
   // Loop over all bank data of this event
@@ -296,35 +268,35 @@ bool MEP::transpose_event(
   // L0Calo doesn't exist in the upgrade
   LHCb::RawBank::BankType prev_type = LHCb::RawBank::L0Calo;
 
-  for (size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block) {
+  for ( size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block ) {
     auto const& [block_header, block_data] = blocks[i_block];
-    auto bank_type = static_cast<LHCb::RawBank::BankType>(block_header->bank_types()[0]);
-    auto& source_offsets = input_offsets[i_block];
+    auto        bank_type                  = static_cast<LHCb::RawBank::BankType>( block_header->bank_types()[0] );
+    auto&       source_offsets             = input_offsets[i_block];
+    auto const* bank_sizes                 = block_header->bank_sizes();
 
     // Check what to do with this bank
-    if (bank_type == LHCb::RawBank::ODIN) {
+    if ( bank_type == LHCb::RawBank::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
-      for (uint16_t i_event = start_event; i_event < end_event; ++i_event) {
-        auto odin_data = reinterpret_cast<unsigned int const*>(block_data.data() + source_offsets[i_event]);
-        event_ids.emplace_back(decode_odin(odin_data));
+      for ( uint16_t i_event = start_event; i_event < end_event; ++i_event ) {
+        auto odin_data = reinterpret_cast<unsigned int const*>( block_data.data() + source_offsets[i_event] );
+        event_ids.emplace_back( decode_odin( odin_data ) );
       }
     }
 
     auto const allen_type = bank_ids[bank_type];
-    if (bank_type >= LHCb::RawBank::LastType || allen_type == -1 || !bank_types.count(BankTypes {allen_type})) {
+    if ( bank_type >= LHCb::RawBank::LastType || allen_type == -1 || !bank_types.count( BankTypes{allen_type} ) ) {
       prev_type = bank_type;
-    }
-    else {
-      if (bank_type != prev_type) {
+    } else {
+      if ( bank_type != prev_type ) {
         bank_index = 1;
-        prev_type = bank_type;
+        prev_type  = bank_type;
       }
 
-      auto allen_type = bank_ids[bank_type];
-      auto& slice = std::get<0>(slices[allen_type][slice_index])[0];
-      auto const& event_offsets = std::get<2>(slices[allen_type][slice_index]);
+      auto        allen_type    = bank_ids[bank_type];
+      auto&       slice         = std::get<0>( slices[allen_type][slice_index] )[0];
+      auto const& event_offsets = std::get<2>( slices[allen_type][slice_index] );
 
-      for (size_t i_event = start_event; i_event < end_event && i_event < block_header->n_banks; ++i_event) {
+      for ( size_t i_event = start_event; i_event < end_event && i_event < block_header->n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
         // - number of banks/offsets
         // - offsets to individual banks
@@ -334,31 +306,31 @@ bool MEP::transpose_event(
 
         // Initialize point to write from offset of previous set
         // All bank offsets are uit32_t so cast to that type
-        auto* banks_write = reinterpret_cast<uint32_t*>(slice.data() + event_offsets[i_event - start_event]);
+        auto* banks_write = reinterpret_cast<uint32_t*>( slice.data() + event_offsets[i_event - start_event] );
 
         // Where to write the offsets
         auto* banks_offsets_write = banks_write + 1;
 
-        if (bank_index == 1) {
+        if ( bank_index == 1 ) {
           // Write the number of banks
-          banks_write[0] = banks_count[bank_type];
+          banks_write[0]         = banks_count[bank_type];
           banks_offsets_write[0] = 0;
         }
 
         // get offset for this bank and store offset for next bank
-        auto offset = banks_offsets_write[bank_index - 1];
-        auto frag_size = block_header->bank_sizes()[i_event];
-        banks_offsets_write[bank_index] = offset + frag_size + sizeof(uint32_t);
+        auto offset                     = banks_offsets_write[bank_index - 1];
+        auto frag_size                  = bank_sizes[i_event];
+        banks_offsets_write[bank_index] = offset + frag_size + sizeof( uint32_t );
 
         // Where to write the bank data itself
         banks_write += preamble_words;
 
         // Write sourceID; offset in 32bit words
-        auto word_offset = offset / sizeof(uint32_t);
+        auto word_offset         = offset / sizeof( uint32_t );
         banks_write[word_offset] = block_header->src_id & 0x7FF;
 
         // Write bank data
-        std::memcpy(banks_write + word_offset + 1, block_data.data() + source_offsets[i_event], frag_size);
+        std::memcpy( banks_write + word_offset + 1, block_data.data() + source_offsets[i_event], frag_size );
       }
 
       ++bank_index;
@@ -368,36 +340,19 @@ bool MEP::transpose_event(
 }
 
 std::tuple<bool, bool, size_t> MEP::transpose_events(
-  Allen::Slices& slices,
-  int const slice_index,
-  std::vector<int> const& bank_ids,
-  std::unordered_set<BankTypes> const& bank_types,
-  std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
-  EventIDs& event_ids,
-  MEP::MEP_header const* mep_header,
-  MEP::Blocks const& blocks,
-  MEP::SourceOffsets const& source_offsets,
-  std::tuple<size_t, size_t> const& interval,
-  bool split_by_run)
-{
+    Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
+    std::unordered_set<BankTypes> const& bank_types, std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    EventIDs& event_ids, MEP::MEP_header const* mep_header, MEP::Blocks const& blocks,
+    MEP::SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
   auto [event_start, event_end] = interval;
 
   bool success = true;
 
-  auto to_transpose = allen_offsets(
-    slices, slice_index, bank_ids, bank_types, banks_count, blocks, source_offsets, interval, split_by_run);
-
-  transpose_event(
-    slices,
-    slice_index,
-    bank_ids,
-    bank_types,
-    banks_count,
-    event_ids,
-    mep_header,
-    blocks,
-    source_offsets,
-    {event_start, event_start + to_transpose});
-
-  return {success, to_transpose != (event_end - event_start), to_transpose};
+  auto to_transpose = allen_offsets( slices, slice_index, bank_ids, bank_types, banks_count, blocks, source_offsets,
+                                     interval, split_by_run );
+
+  transpose_event( slices, slice_index, bank_ids, bank_types, banks_count, event_ids, mep_header, blocks,
+                   source_offsets, {event_start, event_start + to_transpose} );
+
+  return {success, to_transpose != ( event_end - event_start ), to_transpose};
 }
-- 
GitLab


From def7c349fe178d92bbf66c7b3473d1296e6e2e8c Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 9 Sep 2021 21:51:00 +0200
Subject: [PATCH 039/120] Configuration used for HLT1 PRR data throughput tests
 with EB

---
 AllenOnline/options/Allen.py | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 1b564e29c..09b0ca8db 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -63,7 +63,7 @@ if run_online:
 
 allen_conf = AllenConfiguration()
 allen_conf.StopTimeout = 5.
-allen_conf.NThreads = 6
+allen_conf.NThreads = 8
 # Device is a string so the PCI ID can also be given
 # allen_conf.Device = "01:00.0"
 allen_conf.Device = "0"
@@ -85,18 +85,18 @@ else:
 allen_conf.Output = ""
 
 mep_provider = MEPProvider()
-mep_provider.NSlices = 8
+mep_provider.NSlices = 36
 mep_provider.EventsPerSlice = 10000
 mep_provider.OutputLevel = 3
 # Number of MEP buffers and number of transpose/offset threads
-mep_provider.BufferConfig = (4, 4)
+mep_provider.BufferConfig = (12, 8)
 mep_provider.TransposeMEPs = False
 mep_provider.SplitByRun = False
 mep_provider.ThreadPerBuffer = True
 
 if run_online:
     mep_provider.Source = "MBM"
-    mep_provider.MBMCom = "FIFO"
+    mep_provider.MBMComMethod = "FIFO"
     mep_provider.Connections = ["Events_0", "Events_1"]
     mep_provider.Requests = [
         'EvType=1;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
-- 
GitLab


From 6a7c1427dde92da7ecac13f898c8ef3ce22970f0 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 9 Sep 2021 22:08:23 +0200
Subject: [PATCH 040/120] Fix copyright

---
 AllenOnline/AllenOnline/MBMOutput.h            | 10 ++++++++++
 AllenOnline/CMakeLists.txt                     | 10 ++++++++++
 AllenOnline/application/test_host_register.cpp | 10 ++++++++++
 AllenOnline/options/Allen.py                   | 10 ++++++++++
 AllenOnline/src/AllenApplication.cpp           | 10 ++++++++++
 AllenOnline/src/AllenApplication.h             | 10 ++++++++++
 AllenOnline/src/AllenConfiguration.cpp         | 10 ++++++++++
 AllenOnline/src/AllenConfiguration.h           | 10 ++++++++++
 AllenOnline/src/MBMOutput.cpp                  | 10 ++++++++++
 9 files changed, 90 insertions(+)

diff --git a/AllenOnline/AllenOnline/MBMOutput.h b/AllenOnline/AllenOnline/MBMOutput.h
index bf8a417af..eb39265e4 100644
--- a/AllenOnline/AllenOnline/MBMOutput.h
+++ b/AllenOnline/AllenOnline/MBMOutput.h
@@ -1,3 +1,13 @@
+/*****************************************************************************\
+* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
+*                                                                             *
+* This software is distributed under the terms of the GNU General Public      *
+* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
+*                                                                             *
+* In applying this licence, CERN does not waive the privileges and immunities *
+* granted to it by virtue of its status as an Intergovernmental Organization  *
+* or submit itself to any jurisdiction.                                       *
+\*****************************************************************************/
 #pragma once
 
 #include <MBM/bmdef.h>
diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 80015b9f4..dd1aca88d 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -1,3 +1,13 @@
+###############################################################################
+# (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           #
+#                                                                             #
+# This software is distributed under the terms of the GNU General Public      #
+# Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   #
+#                                                                             #
+# In applying this licence, CERN does not waive the privileges and immunities #
+# granted to it by virtue of its status as an Intergovernmental Organization  #
+# or submit itself to any jurisdiction.                                       #
+###############################################################################
 ################################################################################
 # Package: AllenOnline
 ################################################################################
diff --git a/AllenOnline/application/test_host_register.cpp b/AllenOnline/application/test_host_register.cpp
index a5e3f6714..525435585 100644
--- a/AllenOnline/application/test_host_register.cpp
+++ b/AllenOnline/application/test_host_register.cpp
@@ -1,3 +1,13 @@
+/*****************************************************************************\
+* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
+*                                                                             *
+* This software is distributed under the terms of the GNU General Public      *
+* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
+*                                                                             *
+* In applying this licence, CERN does not waive the privileges and immunities *
+* granted to it by virtue of its status as an Intergovernmental Organization  *
+* or submit itself to any jurisdiction.                                       *
+\*****************************************************************************/
 #include <Backend/BackendCommon.h>
 #include <MBM/bmdef.h>
 
diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 09b0ca8db..7432cc61b 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -1,4 +1,14 @@
 #!/usr/bin/env python3
+###############################################################################
+# (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           #
+#                                                                             #
+# This software is distributed under the terms of the GNU General Public      #
+# Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   #
+#                                                                             #
+# In applying this licence, CERN does not waive the privileges and immunities #
+# granted to it by virtue of its status as an Intergovernmental Organization  #
+# or submit itself to any jurisdiction.                                       #
+###############################################################################
 import os
 import json
 from Configurables import LHCbApp, CondDB, ApplicationMgr
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 6106c6e6d..1b74a6807 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -1,3 +1,13 @@
+/*****************************************************************************\
+* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
+*                                                                             *
+* This software is distributed under the terms of the GNU General Public      *
+* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
+*                                                                             *
+* In applying this licence, CERN does not waive the privileges and immunities *
+* granted to it by virtue of its status as an Intergovernmental Organization  *
+* or submit itself to any jurisdiction.                                       *
+\*****************************************************************************/
 #include <dlfcn.h>
 
 #include <iostream>
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 6f48cf321..8f7e6ab18 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -1,3 +1,13 @@
+/*****************************************************************************\
+* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
+*                                                                             *
+* This software is distributed under the terms of the GNU General Public      *
+* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
+*                                                                             *
+* In applying this licence, CERN does not waive the privileges and immunities *
+* granted to it by virtue of its status as an Intergovernmental Organization  *
+* or submit itself to any jurisdiction.                                       *
+\*****************************************************************************/
 #include <dlfcn.h>
 
 #include <iostream>
diff --git a/AllenOnline/src/AllenConfiguration.cpp b/AllenOnline/src/AllenConfiguration.cpp
index 8d3af7610..e14bfc415 100644
--- a/AllenOnline/src/AllenConfiguration.cpp
+++ b/AllenOnline/src/AllenConfiguration.cpp
@@ -1,3 +1,13 @@
+/*****************************************************************************\
+* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
+*                                                                             *
+* This software is distributed under the terms of the GNU General Public      *
+* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
+*                                                                             *
+* In applying this licence, CERN does not waive the privileges and immunities *
+* granted to it by virtue of its status as an Intergovernmental Organization  *
+* or submit itself to any jurisdiction.                                       *
+\*****************************************************************************/
 #include "GaudiKernel/Service.h"
 #include "AllenConfiguration.h"
 
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index d5c4923a0..2888351d1 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -1,3 +1,13 @@
+/*****************************************************************************\
+* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
+*                                                                             *
+* This software is distributed under the terms of the GNU General Public      *
+* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
+*                                                                             *
+* In applying this licence, CERN does not waive the privileges and immunities *
+* granted to it by virtue of its status as an Intergovernmental Organization  *
+* or submit itself to any jurisdiction.                                       *
+\*****************************************************************************/
 #pragma once
 
 #include <GaudiKernel/Service.h>
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index cfc3c6469..233443ae3 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -1,3 +1,13 @@
+/*****************************************************************************\
+* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
+*                                                                             *
+* This software is distributed under the terms of the GNU General Public      *
+* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
+*                                                                             *
+* In applying this licence, CERN does not waive the privileges and immunities *
+* granted to it by virtue of its status as an Intergovernmental Organization  *
+* or submit itself to any jurisdiction.                                       *
+\*****************************************************************************/
 #include <AllenOnline/MBMOutput.h>
 #include <MDF/MDFHeader.h>
 #include <MDF/RawEventHelpers.h>
-- 
GitLab


From 6cc14cab137458cfaf28eef1677b4a34c0d7e764 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 10 Sep 2021 12:57:04 +0200
Subject: [PATCH 041/120] Follow changes in Allen; remove runtime detection
 hacks

---
 AllenOnline/CMakeLists.txt                     | 17 +++--------------
 AllenOnline/application/test_host_register.cpp | 10 ----------
 AllenOnline/src/AllenApplication.cpp           |  1 -
 AllenOnline/src/AllenApplication.h             |  1 -
 4 files changed, 3 insertions(+), 26 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index dd1aca88d..3a6db2311 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -34,23 +34,12 @@ include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
 
 include_directories(include)
 
-# More hacky way to find Allen
-find_library(ALLEN_LIBRARY
-  AllenLib
-  PATH_SUFFIXES ../lib
-  REQUIRED)
-find_package(CUDAToolkit REQUIRED)
-
-set(TARGET_DEVICE "CUDA")
-set(TARGET_DEFINITION "TARGET_DEVICE_${TARGET_DEVICE}")
-add_compile_definitions(${TARGET_DEFINITION})
-
 gaudi_add_library(AllenOnline
                  src/ReadMEP.cpp
                  src/TransposeMEP.cpp
                  PUBLIC_HEADERS AllenOnline
                  INCLUDE_DIRS GaudiKernel cppgsl Online/OnlineKernel Online/EventBuilding
-                 LINK_LIBRARIES GaudiKernel ${ALLEN_LIBRARY} EventBuildingLib CUDA::cudart)
+                 LINK_LIBRARIES GaudiKernel Allen::AllenLib EventBuildingLib)
 
 
 gaudi_add_module(AllenOnlineComp
@@ -60,7 +49,7 @@ gaudi_add_module(AllenOnlineComp
                  src/MBMOutput.cpp
                  INCLUDE_DIRS Online/OnlineBase Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
                  LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQLib RPC Parsers EventBuildingLib MDFLib
-                                OnlineKernel ${ALLEN_LIBRARY} BinaryDumpers DAQEventLib
+                                OnlineKernel Allen::AllenLib BinaryDumpers DAQEventLib
                                 DAQKernelLib GaudiAlgLib PrKernel VPDetLib UTDetLib UTKernelLib AllenOnline
                                 FTDetLib -lrt)
 
@@ -103,4 +92,4 @@ gaudi_add_executable(test_read_mep
 gaudi_add_executable(test_host_register
                      application/test_host_register.cpp
                      INCLUDE_DIRS OnlineBase/OnlineBase
-                     LINK_LIBRARIES GaudiKernel OnlineBase ${ALLEN_LIBRARY} ZMQLib CUDA::cudart)
+                     LINK_LIBRARIES GaudiKernel OnlineBase Allen::Backend ZMQLib)
diff --git a/AllenOnline/application/test_host_register.cpp b/AllenOnline/application/test_host_register.cpp
index 525435585..cd166a592 100644
--- a/AllenOnline/application/test_host_register.cpp
+++ b/AllenOnline/application/test_host_register.cpp
@@ -16,16 +16,6 @@
 #include <chrono>
 #include <iomanip>
 
-#define cudaCheck(stmt)                                                                                            \
-  {                                                                     \
-    cudaError_t err = stmt;                                                                                        \
-    if (err != cudaSuccess) {                                                                                      \
-      fprintf(                                                                                                     \
-        stderr, "Failed to run %s\n%s (%d) at %s: %d\n", #stmt, cudaGetErrorString(err), err, __FILE__, __LINE__); \
-      throw std::invalid_argument("cudaCheck failed");                                                             \
-    }                                                                                                              \
-  }
-
 int main() {
 
   const std::string buffer_name = "Events_0_TDET";
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 1b74a6807..ab8d51206 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -19,7 +19,6 @@
 
 #include <filesystem>
 
-#include <GaudiKernel/IJobOptionsSvc.h>
 #include <GaudiKernel/IMessageSvc.h>
 #include <GaudiKernel/IAppMgrUI.h>
 #include <GaudiKernel/IProperty.h>
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 8f7e6ab18..2006e633b 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -14,7 +14,6 @@
 #include <chrono>
 #include <cmath>
 
-#include <GaudiKernel/IJobOptionsSvc.h>
 #include <GaudiKernel/IMessageSvc.h>
 #include <GaudiKernel/IAppMgrUI.h>
 #include <GaudiKernel/IProperty.h>
-- 
GitLab


From 08a75ca1901d74dda1f50d1adf01c9579a8f7ecb Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 22 Oct 2021 17:44:20 +0200
Subject: [PATCH 042/120] Follow rebase in Allen and changes in Online

---
 AllenOnline/AllenOnline/TransposeMEP.h |  30 ++----
 AllenOnline/application/MPISend.cpp    |  18 ++--
 AllenOnline/application/read_mep.cpp   |  28 +++---
 AllenOnline/include/ReadMEP.h          |   2 +-
 AllenOnline/options/Allen.py           |   3 +-
 AllenOnline/src/AllenApplication.cpp   |   3 +
 AllenOnline/src/AllenConfiguration.h   |   1 +
 AllenOnline/src/MBMOutput.cpp          |   8 +-
 AllenOnline/src/MEPProvider.cpp        |  50 +++++-----
 AllenOnline/src/ReadMEP.cpp            |  24 ++---
 AllenOnline/src/TransposeMEP.cpp       | 123 +++++++++++++------------
 11 files changed, 141 insertions(+), 149 deletions(-)

diff --git a/AllenOnline/AllenOnline/TransposeMEP.h b/AllenOnline/AllenOnline/TransposeMEP.h
index a925dd1cf..c25476900 100644
--- a/AllenOnline/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/AllenOnline/TransposeMEP.h
@@ -34,12 +34,12 @@ namespace {
 namespace MEP {
 
   using SourceOffsets = std::vector<std::vector<uint32_t>>;
-  using Blocks = std::vector<std::tuple<MFP::MFP_header const*, gsl::span<char const>>>;
+  using Blocks = std::vector<EB::MFP const*>;
 
   struct Slice {
-    MEP::MEP_header const* mep_header = nullptr;
-    unsigned packing_factor = 0u;
+    EB::MEP const* mep = nullptr;
     gsl::span<char const> mep_data;
+    unsigned packing_factor = 0u;
     Blocks blocks;
     SourceOffsets offsets;
     size_t slice_size = 0u;
@@ -58,9 +58,9 @@ namespace MEP {
    * @return     (success, number of banks per bank type; 0 if the bank is not needed, bank version per type)
    */
   std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
-  fill_counts(MEP::MEP_header const* header, gsl::span<char const> const& data, std::vector<int> const& bank_ids);
+  fill_counts(EB::MEP const* mep, std::vector<int> const& bank_ids);
 
-  void find_blocks(MEP::MEP_header const* mep_header, gsl::span<char const> const& buffer_span, Blocks& blocks);
+  void find_blocks(EB::MEP const* mep, Blocks& blocks);
 
   void fragment_offsets(Blocks const& blocks, std::vector<std::vector<uint32_t>>& offsets);
 
@@ -104,7 +104,7 @@ namespace MEP {
     std::unordered_set<BankTypes> const& bank_types,
     std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
     EventIDs& event_ids,
-    MEP::MEP_header const* mep_header,
+    EB::MEP const* mep,
     Blocks const& blocks,
     SourceOffsets const& input_offsets,
     std::tuple<size_t, size_t> const& interval);
@@ -127,25 +127,9 @@ namespace MEP {
     std::unordered_set<BankTypes> const& bank_types,
     std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
     EventIDs& event_ids,
-    MEP::MEP_header const* mep_header,
+    EB::MEP const* mep,
     Blocks const& blocks,
     SourceOffsets const& source_offsets,
     std::tuple<size_t, size_t> const& interval,
     bool split_by_run = false);
-
-  /**
-   * @brief      Get MFP header from MEP
-   *
-   * @param      MEP header
-   * @param      index of MFP
-   *
-   * @return     MFP header
-   */
-  inline MFP::MFP_header const* block_header(MEP::MEP_header const* mep_header, size_t block_index)
-  {
-    // Offsets to MFPs are in number of 4-byte words
-    auto const* data = reinterpret_cast<uint32_t const*>(mep_header);
-    auto const offset = mep_header->offsets()[block_index];
-    return reinterpret_cast<MFP::MFP_header const*>(data + offset);
-  }
 } // namespace MEP
diff --git a/AllenOnline/application/MPISend.cpp b/AllenOnline/application/MPISend.cpp
index 9cbbcf1e8..f7f54f342 100644
--- a/AllenOnline/application/MPISend.cpp
+++ b/AllenOnline/application/MPISend.cpp
@@ -80,7 +80,7 @@ int main(int argc, char* argv[]) {
   std::vector<MPI_Request> requests(window_size);
 
   // Read all files in connections
-  std::vector<std::tuple<MEP::MEP_header const*, gsl::span<char>>> meps;
+  std::vector<std::tuple<EB::MEP const*, gsl::span<char const>>> meps;
 
   std::cout << MPI::rank_str() << "Reading "
             << (number_of_meps != 0 ? std::to_string(number_of_meps) : std::string {"all"}) << " meps from files\n";
@@ -93,7 +93,7 @@ int main(int argc, char* argv[]) {
 
   for (const auto& connection : connections) {
     bool eof = false, success = true;
-    MEP::MEP_header const* mep_header = nullptr;
+    EB::MEP const* mep = nullptr;
     unsigned pf = 0;
 
     auto input = LHCb::StreamDescriptor::bind(connection);
@@ -104,7 +104,7 @@ int main(int argc, char* argv[]) {
     while (success && !eof) {
       std::cout << "." << std::flush;
 
-      std::tie(eof, success, mep_header, pf, mep_span) = MEP::read_mep(input, data);
+      std::tie(eof, success, mep, pf, mep_span) = MEP::read_mep(input, data);
       if (!packing_factor) {
         packing_factor = pf;
       }
@@ -122,7 +122,7 @@ int main(int argc, char* argv[]) {
         std::copy_n(mep_span.data(), mep_span.size(), contents);
         ++n_meps_read;
 
-        meps.emplace_back(mep_header, gsl::span<char> {contents, mep_span.size()});
+        meps.emplace_back(mep, gsl::span<char> {contents, mep_span.size()});
       }
       if (n_meps_read >= number_of_meps && number_of_meps != 0) {
         input.close();
@@ -153,12 +153,12 @@ send:
   // MPI: Who am I?
   MPI_Comm_rank(MPI_COMM_WORLD, &MPI::rank);
 
-  auto const& first_header = std::get<0>(*meps.begin());
+  auto const& first_mep = std::get<0>(*meps.begin());
 
   size_t pf = *packing_factor;
   std::cout << "\n"
-            << MPI::rank_str() << "MEP header: " << first_header->n_MFPs << ", " << pf << ", "
-            << first_header->bytes() << "\n";
+            << MPI::rank_str() << "MEP header: " << first_mep->header.n_MFPs << ", " << pf << ", "
+            << first_mep->bytes() << "\n";
   MPI_Send(&pf, 1, MPI_SIZE_T, MPI::receiver, MPI::message::packing_factor, MPI_COMM_WORLD);
 
   MPI_Send(&n_meps_read, 1, MPI_SIZE_T, MPI::receiver, MPI::message::number_of_meps, MPI_COMM_WORLD);
@@ -168,9 +168,9 @@ send:
   while (non_stop || current_mep < meps.size()) {
 
     // Get event data
-    auto const& [mep_header, mep_span] = meps[current_mep];
+    auto const [mep, mep_span] = meps[current_mep];
     const char* current_event_start = mep_span.data();
-    const size_t current_event_size = mep_span.size();
+    const size_t current_event_size = mep_span.size_bytes();
 
     // Notify the event size
     MPI_Send(&current_event_size, 1, MPI_SIZE_T, MPI::receiver, MPI::message::event_size, MPI_COMM_WORLD);
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 63ac75f40..3e93c8440 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -66,26 +66,26 @@ int main(int argc, char* argv[])
 
   for (size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep) {
 
-    std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(input, data);
-    auto const* mep_header = slice.mep_header;
+    std::tie(eof, success, slice.mep, slice.packing_factor, slice.mep_data) = MEP::read_mep(input, data);
+    auto const* mep = slice.mep;
     if (!success) {
       return 1;
     }
     else {
-      cout << "Read mep with packing factor " << slice.packing_factor << " #MFPs: " << mep_header->n_MFPs << "\n";
+      cout << "Read mep with packing factor " << slice.packing_factor << " #MFPs: " << mep->header.n_MFPs << "\n";
     }
 
     if (i_mep == 0) {
       event_ids.reserve(slice.packing_factor);
-      slice.blocks.resize(mep_header->n_MFPs, MEP::Blocks::value_type{});
-      slice.offsets.resize(mep_header->n_MFPs);
+      slice.blocks.resize(mep->header.n_MFPs, MEP::Blocks::value_type{});
+      slice.offsets.resize(mep->header.n_MFPs);
       for (auto& offsets : slice.offsets) {
         offsets.resize(slice.packing_factor + 1);
       }
     }
 
-    MEP::find_blocks(mep_header, slice.mep_data, slice.blocks);
-    auto [sucess, banks_count, banks_version] = MEP::fill_counts(mep_header, slice.mep_data, bank_ids);
+    MEP::find_blocks(mep, slice.blocks);
+    auto [sucess, banks_count, banks_version] = MEP::fill_counts(mep, bank_ids);
 
     if (i_mep == 0) {
       bank_slices = allocate_slices(1, bank_types,
@@ -100,16 +100,16 @@ int main(int argc, char* argv[])
       reset_slice(bank_slices, 0, bank_types, event_ids, true);
     }
 
-    for (auto const& [block_header, block_data] : slice.blocks) {
+    for (auto const* mfp : slice.blocks) {
       // block offsets are in number of 4-byte words
-      auto lhcb_type = int {block_header->bank_types()[0]};
+      auto lhcb_type = int {mfp->header.bank_types()[0]};
 
       cout << "fragment"
-           << " magic: 0x" << std::hex << block_header->magic << std::dec
-           << " packing: " << std::setw(4) << block_header->n_banks << " event_id: " << std::setw(6)
-           << block_header->ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
-           << (block_header->src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{block_header->block_version}
-           << " size: " << std::setw(8) << block_header->bytes() - block_header->header_size() << "\n";
+           << " magic: 0x" << std::hex << mfp->header.magic << std::dec
+           << " packing: " << std::setw(4) << mfp->header.n_banks << " event_id: " << std::setw(6)
+           << mfp->header.ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
+           << (mfp->header.src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{mfp->header.block_version}
+           << " size: " << std::setw(8) << mfp->header.bytes() - mfp->header.header_size() << "\n";
     }
 
     MEP::mep_offsets(bank_slices, 0,
diff --git a/AllenOnline/include/ReadMEP.h b/AllenOnline/include/ReadMEP.h
index 47a1f25cf..03bd59bce 100644
--- a/AllenOnline/include/ReadMEP.h
+++ b/AllenOnline/include/ReadMEP.h
@@ -11,6 +11,6 @@
 #include <MDF/StreamDescriptor.h>
 
 namespace MEP {
-  std::tuple<bool, bool, MEP::MEP_header const*, unsigned, gsl::span<char const>>
+  std::tuple<bool, bool, EB::MEP const*, unsigned, gsl::span<char const>>
   read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer);
 }
diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 7432cc61b..01f3c57fe 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -81,6 +81,7 @@ allen_conf.JSON = json_file
 allen_conf.NLines = n_lines
 allen_conf.OutputChecksum = False
 allen_conf.ParamDir = "/daqarea1/fest/allen/fest_geometry"
+allen_conf.Param = "/group/hlt/fest_202106/dev-dir/Allen/input/parameters"
 allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
 allen_conf.PartitionID = partition_id
@@ -92,7 +93,6 @@ elif integration_test:
     allen_conf.Output = "tcp://192.168.1.101:35000"
 else:
     allen_conf.Output = "allen_output.mdf"
-allen_conf.Output = ""
 
 mep_provider = MEPProvider()
 mep_provider.NSlices = 36
@@ -153,4 +153,3 @@ ApplicationMgr().ExtSvc += [
     'Online::Configuration/Application', 'ZeroMQSvc',
     'MEPProvider'
 ]
-
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index ab8d51206..785e23ba2 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -316,12 +316,14 @@ int AllenApplication::continueProcessing()
 void AllenApplication::allenLoop()
 {
   auto paramDir = resolveEnvVars(m_allenConfig->paramDir);
+  auto param = resolveEnvVars(m_allenConfig->param);
 
   //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
   //--device 23:00.0
   std::map<std::string, std::string> allen_options = {{"v", std::to_string(6 - m_config->outputLevel())},
                                                       {"t", m_allenConfig->nThreads.toString()},
                                                       {"geometry", paramDir},
+						      {"params", param},
                                                       {"configuration", m_json},
                                                       {"device", m_allenConfig->device.value()},
                                                       {"s", std::to_string(m_nSlices)},
@@ -338,6 +340,7 @@ std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
 
   auto output = m_allenConfig->output.value();
   if (output.empty()) {
+    m_logger->warning("No output is configured, selected events will be lost");
     return {};
   }
 
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index 2888351d1..d427e635a 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -36,6 +36,7 @@ public:
   Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json"};
   Gaudi::Property<size_t> nLines{this, "NLines", 0ul};
   Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
+  Gaudi::Property<std::string> param {this, "Param", "${ALLEN_PROJECT_ROOT}/input/parameters"};
 
   Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
   Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 233443ae3..20b3d879f 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -1,3 +1,4 @@
+
 /*****************************************************************************\
 * (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
 *                                                                             *
@@ -59,8 +60,11 @@ bool MBMOutput::start()
   if (m_bmID == MBM_INV_DESC)   {
     return m_logger->error("MBMOutput: failed to connect to MBM buffer %s!",
                            m_bufferName.c_str());
+  } else {
+    m_logger->info("MBMOutput: connected to MBM buffer %s",
+                   m_bufferName.c_str());
+    return true;
   }
-  return true;
 }
 
 bool MBMOutput::write_buffer(size_t)
@@ -81,7 +85,7 @@ bool MBMOutput::write_buffer(size_t)
   long free_len = 0;
 
   auto sc = ::mbm_declare_event(m_bmID, m_buffer.size_bytes(),
-                                EVENT_TYPE_BURST,
+                                EVENT_TYPE_EVENT,
                                 mask, 0, &free_address, &free_len, m_partitionID);
   if (sc == MBM_REQ_CANCEL) {
     return false;
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index a119134be..2da7945a9 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -208,9 +208,8 @@ void MEPProvider::event_sizes(
   for (unsigned int i = 0; i < selected_events.size(); ++i) {
     auto event = selected_events[i];
     sizes[i] +=
-      std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto& entry) {
-        auto const& block_header = std::get<0>(entry);
-        return s + bank_header_size + block_header->bank_sizes()[interval_start + event];
+      std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto* mfp) {
+        return s + bank_header_size + mfp->header.bank_sizes()[interval_start + event];
       });
   }
 }
@@ -227,8 +226,9 @@ void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event,
   size_t offset = 0;
 
   for (size_t i_block = 0; i_block < slice.blocks.size(); ++i_block) {
-    auto const& [block_header, block_data] = slice.blocks[i_block];
-    auto lhcb_type = block_header->bank_types()[0];
+    auto const* mfp = slice.blocks[i_block];
+    auto const* header = &(mfp->header);
+    auto lhcb_type = header->bank_types()[0];
 
     if (prev_type != lhcb_type) {
       block_index = 0;
@@ -239,14 +239,14 @@ void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event,
     // to treat banks needed by Allen and banks not needed by Allen
     // in the same way
     auto const fragment_offset = slice.offsets[i_block][mep_event];
-    auto fragment_size = block_header->bank_sizes()[mep_event];
+    auto fragment_size = header->bank_sizes()[mep_event];
 
     assert((offset + fragment_size) < static_cast<size_t>(buffer.size()));
     offset += add_raw_bank(
-      block_header->bank_types()[mep_event],
-      block_header->block_version,
-      block_header->src_id & 0x7FF,
-      {block_data.data() + fragment_offset, fragment_size},
+      header->bank_types()[mep_event],
+      header->block_version,
+      header->src_id & 0x7FF,
+      {static_cast<char const*>(mfp->payload()) + fragment_offset, fragment_size},
       buffer.data() + offset);
     ++block_index;
   }
@@ -570,8 +570,8 @@ StatusCode MEPProvider::init_mpi()
     Allen::host_register(contents, n_bytes, Allen::hostRegisterDefault);
     m_net_slices[i] = {
       nullptr,
+      {contents, n_bytes},
       0u,
-      gsl::span<char const> {contents, static_cast<events_size>(n_bytes)},
       MEP::Blocks {},
       MEP::SourceOffsets {},
       n_bytes};
@@ -671,8 +671,8 @@ bool MEPProvider::allocate_storage(size_t i_read)
 
   // Offsets are to the start of the event, which includes the header
   auto& slice = m_net_slices[i_read];
-  auto const* mep_header = slice.mep_header;
-  size_t n_blocks = mep_header->n_MFPs;
+  auto const* mep = slice.mep;
+  size_t n_blocks = mep->header.n_MFPs;
 
   if (m_packing_factor == 0) {
     m_packing_factor = slice.packing_factor;
@@ -699,7 +699,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
     }
   }
 
-  std::tie(count_success, m_banks_count, m_banks_version) = MEP::fill_counts(mep_header, slice.mep_data, m_bank_ids);
+  std::tie(count_success, m_banks_count, m_banks_version) = MEP::fill_counts(mep, m_bank_ids);
 
   // Allocate slice memory that will contain transposed banks ready
   // for processing by the Allen kernels
@@ -904,7 +904,7 @@ void MEPProvider::mep_read()
 
     while (!success || eof) {
       if (!m_preload.value() || (preloaded < n_buffers() && !preloaded_buffer[i_buffer])) {
-        std::tie(eof, success, slice.mep_header, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
+        std::tie(eof, success, slice.mep, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
         if (!eof && msgLevel(MSG::DEBUG)) debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
 
         #ifdef HAVE_MPI
@@ -1168,12 +1168,12 @@ void MEPProvider::mpi_read()
     // Wait until all chunks have been sent
     MPI_Waitall(n_sends, requests.data(), MPI_STATUSES_IGNORE);
 
-    slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(contents);
+    slice.mep = reinterpret_cast<EB::MEP const*>(contents);
     slice.mep_data = gsl::span {contents, static_cast<events_size>(mep_size)};
 
-    auto const* mep_header = slice.mep_header;
-    auto const* bh = MEP::block_header(mep_header, 0);
-    slice.packing_factor = bh->n_banks;
+    auto const* mep = slice.mep;
+    auto const* mfp = mep->at(0);
+    slice.packing_factor = mfp->header.n_banks;
 
     if (!m_sizes_known && !allocate_storage(i_buffer)) {
       read_error();
@@ -1337,11 +1337,11 @@ void MEPProvider::bm_read(const std::string& buffer_name)
 
       // info() << "Got MEP " << i_buffer << endmsg;
 
-      slice.mep_header = reinterpret_cast<MEP::MEP_header const*>(ev_data);
-      slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep_header->bytes()};
+      slice.mep = reinterpret_cast<EB::MEP const*>(ev_data);
+      slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep->bytes()};
       slice.slice_size = static_cast<size_t>(ev_len);
-      auto const* block_header = MEP::block_header(slice.mep_header, 0);
-      slice.packing_factor = block_header->n_banks;
+      auto const* mfp = slice.mep->at(0);
+      slice.packing_factor = mfp->header.n_banks;
 
       if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
     } else if (sc == MBM_REQ_CANCEL) {
@@ -1480,7 +1480,7 @@ void MEPProvider::transpose(int thread_id)
     auto& slice = m_net_slices[i_buffer];
 
     // Fill blocks
-    MEP::find_blocks(slice.mep_header, slice.mep_data, slice.blocks);
+    MEP::find_blocks(slice.mep, slice.blocks);
 
     // Fill fragment offsets
     MEP::fragment_offsets(slice.blocks, slice.offsets);
@@ -1495,7 +1495,7 @@ void MEPProvider::transpose(int thread_id)
         m_bank_types,
         m_banks_count,
         event_ids,
-        slice.mep_header,
+        slice.mep,
         slice.blocks,
         slice.offsets,
         interval,
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index 3ec2371d4..c6691471d 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -35,15 +35,15 @@ namespace {
  *
  * @return     (eof, success, mep_header, span of mep data)
  */
-std::tuple<bool, bool, MEP::MEP_header const*, unsigned, gsl::span<char const>>
+std::tuple<bool, bool, EB::MEP const*, unsigned, gsl::span<char const>>
 MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
 {
   // Allocate space for the first few words of the MEP header
-  buffer.resize(sizeof(MEP::MEP_header));
+  buffer.resize(sizeof(EB::MEP_header));
 
   // Read the first few words of the MEP header
   // Why is the number of bytes returned as an int...
-  int read_sc = input.read(&buffer[0], sizeof(MEP::MEP_header));
+  int read_sc = input.read(&buffer[0], sizeof(EB::MEP_header));
   if (read_sc == 0) {
     cout << "Cannot read more data (Header). End-of-File reached.\n";
     return {true, true, nullptr, 0u, {}};
@@ -54,22 +54,22 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   }
 
   // Check magic pattern
-  MEP::MEP_header const* mep_header = reinterpret_cast<MEP::MEP_header const*>(buffer.data());
-  if (mep_header->magic != 0xCEFA) {
-    cerr << "Wrong magic pattern in MEP header: 0x" << std::hex << mep_header->magic << "\n";
+  EB::MEP const* mep = reinterpret_cast<EB::MEP const*>(buffer.data());
+  if (!mep->is_magic_valid()) {
+    cerr << "Wrong magic pattern in MEP header: 0x" << std::hex << mep->header.magic << "\n";
     return {false, false, nullptr, 0u, {}};
   }
 
   // The size of the MEP has been read, so allocate space for that
   // (don't forget to redo the pointers in case the memory was
   // reallocated elsewhere)
-  size_t data_size = mep_header->bytes();
+  size_t data_size = mep->bytes();
   buffer.resize(data_size);
   char* mep_buffer = &buffer[0];
-  mep_header = reinterpret_cast<MEP::MEP_header const*>(mep_buffer);
+  mep = reinterpret_cast<EB::MEP const*>(mep_buffer);
 
-  char* pos = mep_buffer + sizeof(MEP::MEP_header);
-  size_t remaining = data_size - sizeof(MEP::MEP_header);
+  char* pos = mep_buffer + sizeof(EB::MEP_header);
+  size_t remaining = data_size - sizeof(EB::MEP_header);
   while (remaining > 0) {
     size_t chunk = std::min(remaining, static_cast<size_t>(std::numeric_limits<int>::max() / 2));
     // Read the next chunk
@@ -84,8 +84,8 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   }
 
   // Get the packing factor
-  auto const* block_header = reinterpret_cast<MFP::MFP_header const*>(mep_buffer + mep_header->offsets()[0] * sizeof(uint32_t));
+  auto const* mfp = reinterpret_cast<EB::MFP const*>(mep->at(0));
 
-  return {false, true, mep_header, block_header->n_banks,
+  return {false, true, mep, mfp->header.n_banks,
           {buffer.data(), data_size}};
 }
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 04b718ed4..764303112 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -9,68 +9,62 @@
 #include <AllenOnline/TransposeMEP.h>
 
 namespace {
-  std::tuple<unsigned, unsigned long> decode_odin( unsigned const* odin_data ) {
-    unsigned const run_number   = odin_data[LHCb::ODIN::Data::RunNumber];
-    unsigned long  evt_hi       = odin_data[LHCb::ODIN::Data::L0EventIDHi];
-    unsigned long  evt_lo       = odin_data[LHCb::ODIN::Data::L0EventIDLo];
-    unsigned long  event_number = ( evt_hi << 32 ) | evt_lo;
-    return {run_number, event_number};
+  LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes, const uint8_t version ) {
+    LHCb::span<std::uint32_t const> odin_span{reinterpret_cast<std::uint32_t const*>(odin_data + offset), size_bytes / sizeof(uint32_t)};
+    if (version == 6) {
+      return LHCb::ODIN::from_version<6>(odin_span);
+    } else {
+      return LHCb::ODIN{odin_span};
+    }
   }
 } // namespace
 
+
 std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
-MEP::fill_counts( MEP::MEP_header const* header, gsl::span<char const> const& mep_span,
-                  std::vector<int> const& bank_ids ) {
+MEP::fill_counts( EB::MEP const* mep, std::vector<int> const& bank_ids ) {
   // info_cout << "EB header: "
   //   << header.n_blocks << ", "
   //   << header.packing_factor << ", "
   //   << header.reserved << ", "
   //   << header.mep_size << "\n";
 
-  auto                                       header_size = header->header_size();
-  gsl::span<char const>                      block_span{mep_span.data() + header_size, mep_span.size() - header_size};
   std::array<unsigned int, LHCb::NBankTypes> count{0};
   std::array<int, NBankTypes>                versions{0};
-  for ( size_t i = 0; i < header->n_MFPs; ++i ) {
-    auto const* bh = block_header( header, i );
+  for ( size_t i = 0; i < mep->header.n_MFPs; ++i ) {
+    auto const* mfp = mep->at(i);
 
     // info_cout << "EB BlockHeader: "
     //   << bh.event_id << ", " << bh.n_frag << ", " << bh.reserved << ", " << bh.block_size << "\n";
-    auto type = bh->bank_types()[0];
+    auto type = mfp->header.bank_types()[0];
     if ( type < LHCb::RawBank::LastType ) { ++count[type]; }
 
     auto const allen_type = bank_ids[type];
-    if ( allen_type != -1 ) { versions[allen_type] = bh->block_version; }
+    if ( allen_type != -1 ) { versions[allen_type] = mfp->header.block_version; }
   }
 
   return {true, count, versions};
 }
 
-void MEP::find_blocks( MEP::MEP_header const* mep_header, gsl::span<char const> const& buffer_span, Blocks& blocks ) {
+void MEP::find_blocks( EB::MEP const* mep, Blocks& blocks ) {
   // Fill blocks in temporary container
   Blocks tmp{blocks};
-  for ( size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block ) {
+  for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) {
     // block offsets are in 4-byte words with respect to the start of the MEP header
-    auto                   block_offset = mep_header->offsets()[i_block] * sizeof( uint32_t );
-    char const*            block_start  = buffer_span.data() + block_offset;
-    MFP::MFP_header const* block_header = reinterpret_cast<MFP::MFP_header const*>( block_start );
-    gsl::span<char const>  block_data{block_start + block_header->header_size(),
-                                     block_header->bytes() - block_header->header_size()};
-    tmp[i_block] = std::tuple{block_header, std::move( block_data )};
+    tmp[i_block] = mep->at(i_block);
   }
 
-  auto const*         src_ids = mep_header->src_ids();
+  auto const*         src_ids = mep->header.src_ids();
   std::vector<size_t> perm( tmp.size() );
   std::iota( perm.begin(), perm.end(), 0U );
   std::sort( perm.begin(), perm.end(), [&tmp, src_ids]( size_t a, size_t b ) {
     if ( src_ids[a] == src_ids[b] ) {
-      return std::get<0>( tmp[a] )->bank_types()[0] < std::get<0>( tmp[b] )->bank_types()[0];
+      return tmp[a]->header.bank_types()[0] < tmp[b]->header.bank_types()[0];
     } else {
       return src_ids[a] < src_ids[b];
     }
   } );
 
-  for ( size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block ) { blocks[i_block] = tmp[perm[i_block]]; }
+  for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) { blocks[i_block] = tmp[perm[i_block]]; }
 }
 
 void MEP::fragment_offsets( MEP::Blocks const& blocks, MEP::SourceOffsets& offsets ) {
@@ -80,13 +74,15 @@ void MEP::fragment_offsets( MEP::Blocks const& blocks, MEP::SourceOffsets& offse
 
   // Loop over all bank sizes in all blocks
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const& [block_header, block_data] = blocks[i_block];
+    auto const* mfp = blocks[i_block];
+    auto const align = mfp->header.align;
     auto&       o                          = offsets[i_block];
     uint32_t    fragment_offset            = 0;
-    auto const* bank_sizes                 = block_header->bank_sizes();
-    for ( size_t i = 0; i < block_header->n_banks; ++i ) {
+    auto const* bank_sizes                 = mfp->header.bank_sizes();
+    for ( size_t i = 0; i < mfp->header.n_banks; ++i ) {
       o[i] = fragment_offset;
-      fragment_offset += bank_sizes[i];
+      auto const size = bank_sizes[i];
+      fragment_offset += (size + EB::get_padding(size, 1 << align));
     }
   }
 }
@@ -101,19 +97,21 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
 
   // Loop over all bank sizes in all blocks
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const& [block_header, block_data] = blocks[i_block];
-    auto const* bank_sizes                 = block_header->bank_sizes();
-    auto        lhcb_type                  = block_header->bank_types()[0];
-    auto        allen_type                 = bank_ids[lhcb_type];
-    auto&       source_offsets             = input_offsets[i_block];
-    uint        run_number                 = 0;
+    auto const* mfp = blocks[i_block];
+    auto const* bank_sizes     = mfp->header.bank_sizes();
+    auto const  align          = mfp->header.align;
+    auto        lhcb_type      = mfp->header.bank_types()[0];
+    auto        allen_type     = bank_ids[lhcb_type];
+    auto&       source_offsets = input_offsets[i_block];
+    unsigned    run_number     = 0;
     if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
       for ( size_t i = event_start; i < event_end; ++i ) {
         // First check for run changes in ODIN banks
         if ( split_by_run && lhcb_type == LHCb::RawBank::ODIN ) {
           // decode ODIN banks to check for run changes
-          auto           odin_data = reinterpret_cast<unsigned int const*>( block_data.data() + source_offsets[i] );
-          unsigned const odin_run_number = odin_data[LHCb::ODIN::Data::RunNumber];
+          auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), source_offsets[i],
+                                        (bank_sizes[i] / sizeof(std::uint32_t)), mfp->header.block_version);
+          unsigned const odin_run_number = odin.runNumber();
           // if splitting by run, check all events have same run number
           if ( i == event_start ) {
             run_number = odin_run_number;
@@ -127,7 +125,7 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
         auto& event_offsets = std::get<2>( slices[allen_type][slice_index] );
 
         // Allen raw bank format has the sourceID followed by the raw bank data
-        event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i];
+        event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
       }
     }
   }
@@ -180,13 +178,14 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
   size_t        offset_index = 0;
   uint          run_number   = 0;
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const& [block_header, block_data] = blocks[i_block];
+    auto const* mfp = blocks[i_block];
     // FIXME: BankType could be an error bank for that
     // subdetector. Use something better.
-    auto        lhcb_type  = block_header->bank_types()[0];
+    auto        lhcb_type  = mfp->header.bank_types()[0];
     auto        allen_type = bank_ids[lhcb_type];
     auto        n_blocks   = banks_count[lhcb_type];
-    auto const* bank_sizes = block_header->bank_sizes();
+    auto const* bank_sizes = mfp->header.bank_sizes();
+    auto const  align      = mfp->header.align;
 
     // Decode ODIN banks
     if ( lhcb_type == LHCb::RawBank::ODIN ) {
@@ -194,8 +193,10 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       unsigned fragment_offset = 0;
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
         if ( i_event >= event_start ) {
-          auto const* odin_data = reinterpret_cast<unsigned int const*>( block_data.data() + fragment_offset );
-          auto const [odin_run_number, event_number] = decode_odin( odin_data );
+          auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), fragment_offset,
+                                        (bank_sizes[i_event] / sizeof(std::uint32_t)), mfp->header.block_version);
+          unsigned const odin_run_number = odin.runNumber();
+          auto const event_number = odin.eventNumber();
 
           // if splitting by run, check all events have same run number
           if ( i_event == event_start ) {
@@ -206,13 +207,12 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
           }
           event_ids.emplace_back( odin_run_number, event_number );
         }
-        fragment_offset += bank_sizes[i_event];
+        fragment_offset += bank_sizes[i_event] + EB::get_padding(bank_sizes[i_event], 1 << align);
       }
     }
 
     if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
       auto& [spans, data_size, event_offsets, offsets_size] = slices[allen_type][slice_index];
-      auto const* bank_sizes                                = block_header->bank_sizes();
 
       // Calculate block offset and size
       size_t interval_offset = 0, interval_size = 0;
@@ -229,7 +229,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       }
 
       // Store source ID, mask top 5 bits which are detector ID
-      event_offsets[2 + offset_index] = block_header->src_id & 0x7FF;
+      event_offsets[2 + offset_index] = mfp->header.src_id & 0x7FF;
 
       // Initialize the first offsets using the block sizes,
       if ( offset_index < banks_count[lhcb_type] - 1 ) {
@@ -241,13 +241,13 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       for ( size_t i = event_start; i < event_end; ++i ) {
         idx               = i - event_start + 1;
         oi                = 2 + n_blocks * ( 1 + idx ) + offset_index;
-        event_offsets[oi] = event_offsets[oi - n_blocks] + bank_sizes[i];
+        event_offsets[oi] = event_offsets[oi - n_blocks] + bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
       }
       // Update offsets_size
       offsets_size = oi;
 
       // Store block span for this interval
-      spans.emplace_back( const_cast<char*>( block_data.data() ) + interval_offset, interval_size );
+      spans.emplace_back( const_cast<char*>( static_cast<char const*>(mfp->payload()) ) + interval_offset, interval_size );
       data_size += interval_size;
 
       ++offset_index;
@@ -259,7 +259,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
 bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
                            std::unordered_set<BankTypes> const&              bank_types,
                            std::array<unsigned int, LHCb::NBankTypes> const& banks_count, EventIDs& event_ids,
-                           MEP::MEP_header const* mep_header, MEP::Blocks const& blocks,
+                           EB::MEP const* mep, MEP::Blocks const& blocks,
                            MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval ) {
   auto [start_event, end_event] = interval;
 
@@ -268,18 +268,19 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
   // L0Calo doesn't exist in the upgrade
   LHCb::RawBank::BankType prev_type = LHCb::RawBank::L0Calo;
 
-  for ( size_t i_block = 0; i_block < mep_header->n_MFPs; ++i_block ) {
-    auto const& [block_header, block_data] = blocks[i_block];
-    auto        bank_type                  = static_cast<LHCb::RawBank::BankType>( block_header->bank_types()[0] );
-    auto&       source_offsets             = input_offsets[i_block];
-    auto const* bank_sizes                 = block_header->bank_sizes();
+  for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) {
+    auto const* mfp            = blocks[i_block];
+    auto        bank_type      = static_cast<LHCb::RawBank::BankType>( mfp->header.bank_types()[0] );
+    auto&       source_offsets = input_offsets[i_block];
+    auto const* bank_sizes     = mfp->header.bank_sizes();
 
     // Check what to do with this bank
     if ( bank_type == LHCb::RawBank::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
       for ( uint16_t i_event = start_event; i_event < end_event; ++i_event ) {
-        auto odin_data = reinterpret_cast<unsigned int const*>( block_data.data() + source_offsets[i_event] );
-        event_ids.emplace_back( decode_odin( odin_data ) );
+        auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), source_offsets[i_event],
+                                      (bank_sizes[i_event] / sizeof(std::uint32_t)), mfp->header.block_version);
+        event_ids.emplace_back( odin.runNumber(), odin.eventNumber() );
       }
     }
 
@@ -296,7 +297,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
       auto&       slice         = std::get<0>( slices[allen_type][slice_index] )[0];
       auto const& event_offsets = std::get<2>( slices[allen_type][slice_index] );
 
-      for ( size_t i_event = start_event; i_event < end_event && i_event < block_header->n_banks; ++i_event ) {
+      for ( size_t i_event = start_event; i_event < end_event && i_event < mfp->header.n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
         // - number of banks/offsets
         // - offsets to individual banks
@@ -327,10 +328,10 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
         // Write sourceID; offset in 32bit words
         auto word_offset         = offset / sizeof( uint32_t );
-        banks_write[word_offset] = block_header->src_id & 0x7FF;
+        banks_write[word_offset] = mfp->header.src_id & 0x7FF;
 
         // Write bank data
-        std::memcpy( banks_write + word_offset + 1, block_data.data() + source_offsets[i_event], frag_size );
+        std::memcpy( banks_write + word_offset + 1, static_cast<char const*>(mfp->payload()) + source_offsets[i_event], frag_size );
       }
 
       ++bank_index;
@@ -342,7 +343,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 std::tuple<bool, bool, size_t> MEP::transpose_events(
     Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types, std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
-    EventIDs& event_ids, MEP::MEP_header const* mep_header, MEP::Blocks const& blocks,
+    EventIDs& event_ids, EB::MEP const* mep, MEP::Blocks const& blocks,
     MEP::SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
   auto [event_start, event_end] = interval;
 
@@ -351,7 +352,7 @@ std::tuple<bool, bool, size_t> MEP::transpose_events(
   auto to_transpose = allen_offsets( slices, slice_index, bank_ids, bank_types, banks_count, blocks, source_offsets,
                                      interval, split_by_run );
 
-  transpose_event( slices, slice_index, bank_ids, bank_types, banks_count, event_ids, mep_header, blocks,
+  transpose_event( slices, slice_index, bank_ids, bank_types, banks_count, event_ids, mep, blocks,
                    source_offsets, {event_start, event_start + to_transpose} );
 
   return {success, to_transpose != ( event_end - event_start ), to_transpose};
-- 
GitLab


From b1a9578f146822f45ca69deb51487ad299bb9840 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 25 Oct 2021 14:47:01 +0200
Subject: [PATCH 043/120] Add sequence property to AllenConfiguration

---
 AllenOnline/options/Allen.py         | 6 ++++--
 AllenOnline/src/AllenApplication.cpp | 3 ++-
 AllenOnline/src/AllenConfiguration.h | 1 +
 3 files changed, 7 insertions(+), 3 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 01f3c57fe..8f7d84efa 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -47,7 +47,8 @@ CondDB().Upgrade = True
 # Get the number of lines from the configuration. This should probably
 # be done internally to Allen and retrieved somehow
 n_lines = 0
-json_file = "/daqarea1/fest/allen/configuration/passthrough.json"
+# json_file = "/daqarea1/fest/allen/configuration/passthrough.json"
+json_file = "/group/hlt/fest_202106/dev-dir/MooreOnline/AllenOnline/options/passthrough.json"
 with open(json_file) as jf:
     json_config = json.load(jf)
     gs = json_config.get('gather_selections', {})
@@ -78,6 +79,7 @@ allen_conf.NThreads = 8
 # allen_conf.Device = "01:00.0"
 allen_conf.Device = "0"
 allen_conf.JSON = json_file
+allen_conf.Sequence = "passthrough"
 allen_conf.NLines = n_lines
 allen_conf.OutputChecksum = False
 allen_conf.ParamDir = "/daqarea1/fest/allen/fest_geometry"
@@ -137,7 +139,7 @@ app = ApplicationMgr()
 app.MessageSvcType = 'MessageSvc'
 app.OutputLevel = output_level
 messageSvc = MessageSvc('MessageSvc')
-messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M\n'
+messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
 messageSvc.OutputLevel = output_level
 
 
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 785e23ba2..97e217f9e 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -327,7 +327,8 @@ void AllenApplication::allenLoop()
                                                       {"configuration", m_json},
                                                       {"device", m_allenConfig->device.value()},
                                                       {"s", std::to_string(m_nSlices)},
-                                                      {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())}};
+                                                      {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())},
+                                                      {"sequence", m_allenConfig->sequence.value()}};
   allen(allen_options, m_updater, m_provider, m_output.get(), m_zmqSvc.get(), m_controlConnection);
 }
 
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index d427e635a..e67cf1446 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -37,6 +37,7 @@ public:
   Gaudi::Property<size_t> nLines{this, "NLines", 0ul};
   Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
   Gaudi::Property<std::string> param {this, "Param", "${ALLEN_PROJECT_ROOT}/input/parameters"};
+  Gaudi::Property<std::string> sequence {this, "Sequence", "hlt1_pp_default"};
 
   Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
   Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
-- 
GitLab


From cd49c29686edb975952d9fa07539a2bad2326ce3 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 25 Oct 2021 14:47:29 +0200
Subject: [PATCH 044/120] Remove spurious division by sizeof(uint32_t)

---
 AllenOnline/src/TransposeMEP.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 764303112..7bd53193d 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -110,7 +110,7 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
         if ( split_by_run && lhcb_type == LHCb::RawBank::ODIN ) {
           // decode ODIN banks to check for run changes
           auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), source_offsets[i],
-                                        (bank_sizes[i] / sizeof(std::uint32_t)), mfp->header.block_version);
+                                        bank_sizes[i], mfp->header.block_version);
           unsigned const odin_run_number = odin.runNumber();
           // if splitting by run, check all events have same run number
           if ( i == event_start ) {
@@ -194,7 +194,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
         if ( i_event >= event_start ) {
           auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), fragment_offset,
-                                        (bank_sizes[i_event] / sizeof(std::uint32_t)), mfp->header.block_version);
+                                        bank_sizes[i_event], mfp->header.block_version);
           unsigned const odin_run_number = odin.runNumber();
           auto const event_number = odin.eventNumber();
 
-- 
GitLab


From 6f941487f3e8bf2175bfc7b12d31cf8b7d28b3d8 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 25 Oct 2021 17:24:26 +0200
Subject: [PATCH 045/120] Fix wrong initialization of std::optional

---
 AllenOnline/src/MEPProvider.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 2da7945a9..62847f467 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -302,7 +302,7 @@ StatusCode MEPProvider::initialize()
   }
   #endif
 
-  std::optional<size_t> n_events;
+  std::optional<size_t> n_events = std::nullopt;
   if (m_nevents.value() >= 0)
     n_events = static_cast<size_t>(m_nevents.value());
 
-- 
GitLab


From 263b236f8ebb1af2109d69443493aceb28248c1f Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 26 Oct 2021 11:46:11 +0200
Subject: [PATCH 046/120] Explicitly initialize std::optional

---
 AllenOnline/AllenOnline/MEPProvider.h | 6 +++---
 AllenOnline/application/MPISend.cpp   | 3 ++-
 AllenOnline/src/MEPProvider.cpp       | 2 +-
 3 files changed, 6 insertions(+), 5 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 5f811a3fd..21659798b 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -101,7 +101,7 @@ public:
    *
    * @return     EventIDs of events in given slice
    */
-  EventIDs event_ids(size_t slice_index, std::optional<size_t> first = {}, std::optional<size_t> last = {})
+  EventIDs event_ids(size_t slice_index, std::optional<size_t> first = std::nullopt, std::optional<size_t> last = std::nullopt)
     const override;
 
   /**
@@ -122,7 +122,7 @@ public:
    * @return     (good slice, timed out, slice index, number of events in slice)
    */
   std::tuple<bool, bool, bool, size_t, size_t, uint> get_slice(
-    std::optional<unsigned int> timeout = {}) override;
+    std::optional<unsigned int> timeout = std::nullopt) override;
 
   /**
    * @brief      Declare a slice free for reuse; thread-safe
@@ -267,7 +267,7 @@ private:
   std::vector<EventIDs> m_event_ids;
 
   // Storage for the currently open input file
-  mutable std::optional<LHCb::StreamDescriptor::Access> m_input;
+  mutable std::optional<LHCb::StreamDescriptor::Access> m_input = std::nullopt;
 
   // Iterator that points to the filename of the currently open file
   mutable std::vector<std::string>::const_iterator m_current;
diff --git a/AllenOnline/application/MPISend.cpp b/AllenOnline/application/MPISend.cpp
index f7f54f342..c125981ce 100644
--- a/AllenOnline/application/MPISend.cpp
+++ b/AllenOnline/application/MPISend.cpp
@@ -5,6 +5,7 @@
 #include <string>
 #include <iostream>
 #include <vector>
+#include <optional>
 
 #include <sys/types.h>
 #include <sys/stat.h>
@@ -89,7 +90,7 @@ int main(int argc, char* argv[]) {
   gsl::span<char const> mep_span;
   size_t n_meps_read = 0;
 
-  std::optional<unsigned> packing_factor;
+  std::optional<unsigned> packing_factor = std::nullopt;
 
   for (const auto& connection : connections) {
     bool eof = false, success = true;
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 62847f467..4af1dd1e0 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -1403,7 +1403,7 @@ void MEPProvider::transpose(int thread_id)
 
   size_t i_buffer = 0;
   std::tuple<size_t, size_t> interval;
-  std::optional<size_t> slice_index;
+  std::optional<size_t> slice_index = std::nullopt;
 
   bool good = false, transpose_full = false;
   size_t n_transposed = 0;
-- 
GitLab


From 6927d4a36df69e0e4d0bf7336ef4ad44a25304d9 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 26 Oct 2021 13:27:40 +0200
Subject: [PATCH 047/120] Fix setting of BankTypes property of MEPProvider

---
 AllenOnline/AllenOnline/MEPProvider.h | 1 +
 1 file changed, 1 insertion(+)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 21659798b..01881e7c2 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -293,6 +293,7 @@ private:
                      LHCb::RawBank::EcalPacked,
                      LHCb::RawBank::HcalPacked},
     [this](auto &) -> void {
+      m_bank_types.clear();
       for (auto bt : m_lhcb_bank_types) {
         auto it = Allen::bank_types.find(bt);
         if (it == Allen::bank_types.end()) {
-- 
GitLab


From 180b8cffc9250689a455867cc24af6f69a78fbb5 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 26 Oct 2021 17:44:09 +0200
Subject: [PATCH 048/120] Fix stop->start->stop->start etc.

The stop/start code should be consolidated across different input
methods, i.e. files, MBM and MPI. There may very well be a regression
in file mode. To be tested ASAP
---
 AllenOnline/src/MEPProvider.cpp | 31 ++++++++++++++++++-------------
 1 file changed, 18 insertions(+), 13 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 4af1dd1e0..b7bdc7125 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -173,8 +173,8 @@ void MEPProvider::slice_free(size_t slice_index)
             m_buffer_event[i_buffer] = false;
           }
         }
-        if (msgLevel(MSG::VERBOSE)) {
-          verbose() << "Freed MEP buffer " << i_buffer << endmsg;
+        if (msgLevel(MSG::DEBUG)) {
+          debug() << "Freed MEP buffer " << i_buffer << "; writable: " << count_writable() << endmsg;
           for (auto const& status :  m_buffer_status) {
             verbose() << std::setw(4) << status.index << std::setw(3) << status.writable << std::setw(4) << status.work_counter;
             for (auto interval : status.intervals) {
@@ -356,6 +356,7 @@ StatusCode MEPProvider::start()
 
   std::unique_lock<std::mutex> lock {m_control_mutex};
   m_stopping = false;
+  m_done = false;
 
   // start MPI receive, MEP reading thread or BM thread
   if (m_source == MEP::ProviderSource::MPI && m_input_threads.empty()) {
@@ -415,15 +416,23 @@ StatusCode MEPProvider::stop()
   }
 
   if (m_source == MEP::ProviderSource::MBM) {
-    // Cancel all requests to the buffer manager for those who are waiting
-    for (size_t b = 0; b < m_buffer_status.size(); ++b) {
-      if (!m_buffer_status[b].writable) ::mbm_cancel_request(m_bmIDs[b]);
+    {
+      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+      debug() << "Cancelling MBM requests; n_writable: " << count_writable() << endmsg;
+      // Cancel all requests to the buffer manager for those who are waiting
+      for (size_t b = 0; b < m_buffer_status.size(); ++b) {
+        if (!m_buffer_status[b].writable) ::mbm_cancel_request(m_bmIDs[b]);
+      }
     }
 
     for (auto& input_thread : m_input_threads) {
       input_thread.join();
     }
     m_input_threads.clear();
+
+    m_done = true;
+    debug() << "Prefetch notifying all" << endmsg;
+    m_transpose_cond.notify_all();
   }
 
   {
@@ -1301,6 +1310,9 @@ void MEPProvider::bm_read(const std::string& buffer_name)
       }
     }
     if (m_done) {
+      if (buffer_reading != m_buffer_status.end()) {
+        buffer_reading->writable = true;
+      }
       break;
     }
 
@@ -1349,9 +1361,6 @@ void MEPProvider::bm_read(const std::string& buffer_name)
       m_buffer_status[i_buffer].writable = true;
       cancelled = true;
       if (msgLevel(MSG::DEBUG)) debug() << "Got cancel" << endmsg;
-      if (count_writable() == n_buffers()) {
-        m_done = true;
-      }
     }
 
     if (!m_sizes_known && !cancelled) {
@@ -1377,11 +1386,7 @@ void MEPProvider::bm_read(const std::string& buffer_name)
       set_intervals(status.intervals, size_t {slice.packing_factor});
     }
 
-    if (m_done) {
-      debug() << "Prefetch notifying all" << endmsg;
-      m_transpose_cond.notify_all();
-    }
-    else if (!cancelled) {
+    if (!cancelled) {
       debug() << "Prefetch notifying one" << endmsg;
       m_transpose_cond.notify_one();
     } else if (cancelled) {
-- 
GitLab


From ca9431d814994a44a5db7bebd994513b9a8c53fb Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 26 Oct 2021 17:51:54 +0200
Subject: [PATCH 049/120] Remove find_package(CUDAToolkit) from AllenOnline

This is handled by AllenDependencies
---
 AllenOnline/CMakeLists.txt | 1 -
 1 file changed, 1 deletion(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 3a6db2311..0195ee4c8 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -26,7 +26,6 @@ gaudi_depends_on_subdirs(Online/OnlineKernel
 find_package(cppgsl REQUIRED)
 find_package(ZMQ REQUIRED)
 find_package(Boost REQUIRED COMPONENTS program_options)
-find_package(CUDAToolkit REQUIRED)
 
 include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
   ${ZMQ_INCLUDE_DIRS}
-- 
GitLab


From 452d460876efc11ebd9f33dd4d3410eebdab4c6a Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 27 Oct 2021 22:59:22 +0200
Subject: [PATCH 050/120] Fix padding in output raw banks.

Also cache some pointers to the content of MFPs to avoid expensive
calls later.
---
 AllenOnline/AllenOnline/TransposeMEP.h | 14 +++++-
 AllenOnline/application/read_mep.cpp   | 23 ++++++----
 AllenOnline/src/MEPProvider.cpp        | 33 ++++++--------
 AllenOnline/src/TransposeMEP.cpp       | 59 +++++++++++++-------------
 AllenOnline/src/WriteMDF.h             | 25 -----------
 5 files changed, 69 insertions(+), 85 deletions(-)
 delete mode 100644 AllenOnline/src/WriteMDF.h

diff --git a/AllenOnline/AllenOnline/TransposeMEP.h b/AllenOnline/AllenOnline/TransposeMEP.h
index c25476900..d36743777 100644
--- a/AllenOnline/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/AllenOnline/TransposeMEP.h
@@ -34,7 +34,19 @@ namespace {
 namespace MEP {
 
   using SourceOffsets = std::vector<std::vector<uint32_t>>;
-  using Blocks = std::vector<EB::MFP const*>;
+
+  struct Block {
+    Block() = default;
+    Block(EB::MFP const* b) : mfp{b}, header{&(mfp->header)}, bank_sizes{mfp->header.bank_sizes()}, bank_types{mfp->header.bank_types()}, payload{static_cast<char const*>(mfp->payload())} {}
+
+    EB::MFP const* mfp = nullptr;
+    EB::MFP_header const* header = nullptr;
+    EB::bank_size_type const* bank_sizes = nullptr;
+    EB::bank_type_type const* bank_types = nullptr;
+    char const* payload = nullptr;
+  };
+
+  using Blocks = std::vector<Block>;
 
   struct Slice {
     EB::MEP const* mep = nullptr;
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 3e93c8440..a1d2c98b7 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -62,7 +62,7 @@ int main(int argc, char* argv[])
 
   Allen::Slices bank_slices;
 
-  std::unordered_set bank_types = {BankTypes::ODIN};
+  std::unordered_set bank_types = {BankTypes::ODIN, BankTypes::Rich};
 
   for (size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep) {
 
@@ -100,16 +100,23 @@ int main(int argc, char* argv[])
       reset_slice(bank_slices, 0, bank_types, event_ids, true);
     }
 
-    for (auto const* mfp : slice.blocks) {
+    for (auto const& block : slice.blocks) {
       // block offsets are in number of 4-byte words
-      auto lhcb_type = int {mfp->header.bank_types()[0]};
+      auto lhcb_type = int {block.bank_types[0]};
+
+      size_t padded_size = 0;
+      auto const* bank_sizes = block.bank_sizes;
+      for (size_t i = 0; i < block.header->n_banks; ++i) {
+        padded_size += bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << block.header->align);
+      }
 
       cout << "fragment"
-           << " magic: 0x" << std::hex << mfp->header.magic << std::dec
-           << " packing: " << std::setw(4) << mfp->header.n_banks << " event_id: " << std::setw(6)
-           << mfp->header.ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
-           << (mfp->header.src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{mfp->header.block_version}
-           << " size: " << std::setw(8) << mfp->header.bytes() - mfp->header.header_size() << "\n";
+           << " magic: 0x" << std::hex << block.header->magic << std::dec
+           << " packing: " << std::setw(4) << block.header->n_banks << " event_id: " << std::setw(6)
+           << block.header->ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
+           << (block.header->src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{block.header->block_version}
+           << " size: " << std::setw(8) << block.header->bytes() - block.header->header_size()
+           << " padded_size: " << std::setw(8) << padded_size << "\n";
     }
 
     MEP::mep_offsets(bank_slices, 0,
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index b7bdc7125..9ea9b1e6d 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -8,6 +8,7 @@
 #include <Allen/BankMapping.h>
 #include <Allen/Timer.h>
 #include <Allen/SliceUtils.h>
+#include <Allen/write_mdf.hpp>
 #include <Backend/BackendCommon.h>
 
 #include <MDF/StreamDescriptor.h>
@@ -22,7 +23,6 @@
 
 #include "AllenConfiguration.h"
 
-#include "WriteMDF.h"
 #include "ReadMEP.h"
 
 #ifdef HAVE_MPI
@@ -208,8 +208,9 @@ void MEPProvider::event_sizes(
   for (unsigned int i = 0; i < selected_events.size(); ++i) {
     auto event = selected_events[i];
     sizes[i] +=
-      std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto* mfp) {
-        return s + bank_header_size + mfp->header.bank_sizes()[interval_start + event];
+      std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto& block) {
+        auto const fragment_size = block.bank_sizes[interval_start + event];
+        return s + bank_header_size + Allen::padded_bank_size(fragment_size);
       });
   }
 }
@@ -220,35 +221,25 @@ void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event,
   const auto mep_event = interval_start + event;
 
   auto const& slice = m_net_slices[i_buffer];
-
-  unsigned char prev_type = 0;
-  auto block_index = 0;
   size_t offset = 0;
 
   for (size_t i_block = 0; i_block < slice.blocks.size(); ++i_block) {
-    auto const* mfp = slice.blocks[i_block];
-    auto const* header = &(mfp->header);
-    auto lhcb_type = header->bank_types()[0];
-
-    if (prev_type != lhcb_type) {
-      block_index = 0;
-      prev_type = lhcb_type;
-    }
+    auto const& block = slice.blocks[i_block];
+    auto const* header = block.header;
 
     // All banks are taken directly from the block data to be able
     // to treat banks needed by Allen and banks not needed by Allen
     // in the same way
     auto const fragment_offset = slice.offsets[i_block][mep_event];
-    auto fragment_size = header->bank_sizes()[mep_event];
+    auto fragment_size = block.bank_sizes[mep_event];
 
-    assert((offset + fragment_size) < static_cast<size_t>(buffer.size()));
-    offset += add_raw_bank(
-      header->bank_types()[mep_event],
+    assert((offset + bank_header_size + Allen::padded_bank_size(fragment_size)) < static_cast<size_t>(buffer.size()));
+    offset += Allen::add_raw_bank(
+      block.bank_types[mep_event],
       header->block_version,
-      header->src_id & 0x7FF,
-      {static_cast<char const*>(mfp->payload()) + fragment_offset, fragment_size},
+      header->src_id,
+      {block.payload + fragment_offset, fragment_size},
       buffer.data() + offset);
-    ++block_index;
   }
 }
 
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 7bd53193d..921d0a771 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -50,7 +50,7 @@ void MEP::find_blocks( EB::MEP const* mep, Blocks& blocks ) {
   Blocks tmp{blocks};
   for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) {
     // block offsets are in 4-byte words with respect to the start of the MEP header
-    tmp[i_block] = mep->at(i_block);
+    tmp[i_block] = MEP::Block{mep->at(i_block)};
   }
 
   auto const*         src_ids = mep->header.src_ids();
@@ -58,7 +58,7 @@ void MEP::find_blocks( EB::MEP const* mep, Blocks& blocks ) {
   std::iota( perm.begin(), perm.end(), 0U );
   std::sort( perm.begin(), perm.end(), [&tmp, src_ids]( size_t a, size_t b ) {
     if ( src_ids[a] == src_ids[b] ) {
-      return tmp[a]->header.bank_types()[0] < tmp[b]->header.bank_types()[0];
+      return tmp[a].bank_types[0] < tmp[b].bank_types[0];
     } else {
       return src_ids[a] < src_ids[b];
     }
@@ -74,14 +74,13 @@ void MEP::fragment_offsets( MEP::Blocks const& blocks, MEP::SourceOffsets& offse
 
   // Loop over all bank sizes in all blocks
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const* mfp = blocks[i_block];
-    auto const align = mfp->header.align;
+    auto const& block = blocks[i_block];
+    auto const align = block.header->align;
     auto&       o                          = offsets[i_block];
     uint32_t    fragment_offset            = 0;
-    auto const* bank_sizes                 = mfp->header.bank_sizes();
-    for ( size_t i = 0; i < mfp->header.n_banks; ++i ) {
+    for ( size_t i = 0; i < block.header->n_banks; ++i ) {
       o[i] = fragment_offset;
-      auto const size = bank_sizes[i];
+      auto const size = block.bank_sizes[i];
       fragment_offset += (size + EB::get_padding(size, 1 << align));
     }
   }
@@ -97,10 +96,10 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
 
   // Loop over all bank sizes in all blocks
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const* mfp = blocks[i_block];
-    auto const* bank_sizes     = mfp->header.bank_sizes();
-    auto const  align          = mfp->header.align;
-    auto        lhcb_type      = mfp->header.bank_types()[0];
+    auto const& block = blocks[i_block];
+    auto const* bank_sizes     = block.bank_sizes;
+    auto const  align          = block.header->align;
+    auto        lhcb_type      = block.bank_types[0];
     auto        allen_type     = bank_ids[lhcb_type];
     auto&       source_offsets = input_offsets[i_block];
     unsigned    run_number     = 0;
@@ -109,8 +108,8 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
         // First check for run changes in ODIN banks
         if ( split_by_run && lhcb_type == LHCb::RawBank::ODIN ) {
           // decode ODIN banks to check for run changes
-          auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), source_offsets[i],
-                                        bank_sizes[i], mfp->header.block_version);
+          auto const odin = decode_odin(block.payload, source_offsets[i],
+                                        bank_sizes[i], block.header->block_version);
           unsigned const odin_run_number = odin.runNumber();
           // if splitting by run, check all events have same run number
           if ( i == event_start ) {
@@ -178,14 +177,14 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
   size_t        offset_index = 0;
   uint          run_number   = 0;
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const* mfp = blocks[i_block];
+    auto const& block = blocks[i_block];
     // FIXME: BankType could be an error bank for that
     // subdetector. Use something better.
-    auto        lhcb_type  = mfp->header.bank_types()[0];
+    auto        lhcb_type  = block.bank_types[0];
     auto        allen_type = bank_ids[lhcb_type];
     auto        n_blocks   = banks_count[lhcb_type];
-    auto const* bank_sizes = mfp->header.bank_sizes();
-    auto const  align      = mfp->header.align;
+    auto const* bank_sizes = block.bank_sizes;
+    auto const  align      = block.header->align;
 
     // Decode ODIN banks
     if ( lhcb_type == LHCb::RawBank::ODIN ) {
@@ -193,8 +192,8 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       unsigned fragment_offset = 0;
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
         if ( i_event >= event_start ) {
-          auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), fragment_offset,
-                                        bank_sizes[i_event], mfp->header.block_version);
+          auto const odin = decode_odin(block.payload, fragment_offset,
+                                        bank_sizes[i_event], block.header->block_version);
           unsigned const odin_run_number = odin.runNumber();
           auto const event_number = odin.eventNumber();
 
@@ -229,7 +228,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       }
 
       // Store source ID, mask top 5 bits which are detector ID
-      event_offsets[2 + offset_index] = mfp->header.src_id & 0x7FF;
+      event_offsets[2 + offset_index] = block.header->src_id & 0x7FF;
 
       // Initialize the first offsets using the block sizes,
       if ( offset_index < banks_count[lhcb_type] - 1 ) {
@@ -247,7 +246,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       offsets_size = oi;
 
       // Store block span for this interval
-      spans.emplace_back( const_cast<char*>( static_cast<char const*>(mfp->payload()) ) + interval_offset, interval_size );
+      spans.emplace_back( const_cast<char*>( block.payload ) + interval_offset, interval_size );
       data_size += interval_size;
 
       ++offset_index;
@@ -269,17 +268,17 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
   LHCb::RawBank::BankType prev_type = LHCb::RawBank::L0Calo;
 
   for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) {
-    auto const* mfp            = blocks[i_block];
-    auto        bank_type      = static_cast<LHCb::RawBank::BankType>( mfp->header.bank_types()[0] );
+    auto const& block          = blocks[i_block];
+    auto        bank_type      = static_cast<LHCb::RawBank::BankType>( block.bank_types[0] );
     auto&       source_offsets = input_offsets[i_block];
-    auto const* bank_sizes     = mfp->header.bank_sizes();
+    auto const* bank_sizes     = block.bank_sizes;
 
     // Check what to do with this bank
     if ( bank_type == LHCb::RawBank::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
       for ( uint16_t i_event = start_event; i_event < end_event; ++i_event ) {
-        auto const odin = decode_odin(static_cast<char const*>(mfp->payload()), source_offsets[i_event],
-                                      (bank_sizes[i_event] / sizeof(std::uint32_t)), mfp->header.block_version);
+        auto const odin = decode_odin(block.payload, source_offsets[i_event],
+                                      (bank_sizes[i_event] / sizeof(std::uint32_t)), block.header->block_version);
         event_ids.emplace_back( odin.runNumber(), odin.eventNumber() );
       }
     }
@@ -297,7 +296,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
       auto&       slice         = std::get<0>( slices[allen_type][slice_index] )[0];
       auto const& event_offsets = std::get<2>( slices[allen_type][slice_index] );
 
-      for ( size_t i_event = start_event; i_event < end_event && i_event < mfp->header.n_banks; ++i_event ) {
+      for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
         // - number of banks/offsets
         // - offsets to individual banks
@@ -320,7 +319,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
         // get offset for this bank and store offset for next bank
         auto offset                     = banks_offsets_write[bank_index - 1];
-        auto frag_size                  = bank_sizes[i_event];
+        auto frag_size                  = bank_sizes[i_event] + EB::get_padding(bank_sizes[i_event], sizeof(unsigned));
         banks_offsets_write[bank_index] = offset + frag_size + sizeof( uint32_t );
 
         // Where to write the bank data itself
@@ -328,10 +327,10 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
         // Write sourceID; offset in 32bit words
         auto word_offset         = offset / sizeof( uint32_t );
-        banks_write[word_offset] = mfp->header.src_id & 0x7FF;
+        banks_write[word_offset] = block.header->src_id & 0x7FF;
 
         // Write bank data
-        std::memcpy( banks_write + word_offset + 1, static_cast<char const*>(mfp->payload()) + source_offsets[i_event], frag_size );
+        std::memcpy( banks_write + word_offset + 1, block.payload + source_offsets[i_event], frag_size );
       }
 
       ++bank_index;
diff --git a/AllenOnline/src/WriteMDF.h b/AllenOnline/src/WriteMDF.h
deleted file mode 100644
index ebf5d08db..000000000
--- a/AllenOnline/src/WriteMDF.h
+++ /dev/null
@@ -1,25 +0,0 @@
-/*****************************************************************************\
-* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
-\*****************************************************************************/
-#include <cstring>
-
-#include <gsl/gsl>
-
-#include <Event/RawBank.h>
-
-inline size_t add_raw_bank(
-  unsigned char const type,
-  unsigned char const version,
-  short const sourceID,
-  gsl::span<char const> fragment,
-  char* buffer)
-{
-  auto* bank = reinterpret_cast<LHCb::RawBank*>(buffer);
-  bank->setMagic();
-  bank->setSize(fragment.size());
-  bank->setType(static_cast<LHCb::RawBank::BankType>(type));
-  bank->setVersion(version);
-  bank->setSourceID(sourceID);
-  std::memcpy(bank->begin<char>(), fragment.data(), fragment.size());
-  return bank->size() + bank->hdrSize();
-}
-- 
GitLab


From 4cd1420e07f3cd97f3488409a478e3a0d7875342 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 27 Oct 2021 23:13:11 +0200
Subject: [PATCH 051/120] Don't repeatedly cancel

---
 AllenOnline/src/MBMOutput.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 20b3d879f..054cfdef9 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -104,7 +104,7 @@ bool MBMOutput::write_buffer(size_t)
 }
 
 void MBMOutput::cancel() {
-  if (m_bmID != MBM_INV_DESC) {
+  if (!m_cancelled && m_bmID != MBM_INV_DESC) {
     ::mbm_cancel_request(m_bmID);
   }
   m_cancelled = true;
-- 
GitLab


From a82c33ad7ca88452b2eab21550728f187d94e64b Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 28 Oct 2021 14:42:21 +0200
Subject: [PATCH 052/120] Fix assert

---
 AllenOnline/src/MEPProvider.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 9ea9b1e6d..663136cfe 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -233,7 +233,7 @@ void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event,
     auto const fragment_offset = slice.offsets[i_block][mep_event];
     auto fragment_size = block.bank_sizes[mep_event];
 
-    assert((offset + bank_header_size + Allen::padded_bank_size(fragment_size)) < static_cast<size_t>(buffer.size()));
+    assert((offset + bank_header_size + Allen::padded_bank_size(fragment_size)) <= static_cast<size_t>(buffer.size()));
     offset += Allen::add_raw_bank(
       block.bank_types[mep_event],
       header->block_version,
-- 
GitLab


From b0d13d2e44d3f3b526952dc7b49ac3ad8fc97444 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 29 Oct 2021 12:09:56 +0200
Subject: [PATCH 053/120] Follow output changes in Allen

---
 AllenOnline/AllenOnline/MBMOutput.h |  3 ---
 AllenOnline/src/MBMOutput.cpp       | 12 ++----------
 2 files changed, 2 insertions(+), 13 deletions(-)

diff --git a/AllenOnline/AllenOnline/MBMOutput.h b/AllenOnline/AllenOnline/MBMOutput.h
index eb39265e4..3def45847 100644
--- a/AllenOnline/AllenOnline/MBMOutput.h
+++ b/AllenOnline/AllenOnline/MBMOutput.h
@@ -56,9 +56,6 @@ private:
   // partition ID
   unsigned m_partitionID = 0;
 
-  // do checksum on write
-  bool const m_checksum;
-
   // MBM buffer
   gsl::span<char> m_buffer;
 
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 054cfdef9..3fef3c39d 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -21,12 +21,11 @@ MBMOutput::MBMOutput(
     unsigned partitionID,
     size_t const nLines,
     bool const checksum) :
-    OutputHandler {inputProvider, bufferName, nLines},
+    OutputHandler {inputProvider, bufferName, nLines, checksum},
     m_logger{std::move(logger)},
     m_bufferName {std::move(bufferName)},
     m_processName{std::move(processName)},
-    m_partitionID{partitionID},
-    m_checksum{checksum}
+    m_partitionID{partitionID}
 {
 }
 
@@ -73,13 +72,6 @@ bool MBMOutput::write_buffer(size_t)
     return false;
   }
 
-  if (m_checksum) {
-    auto* header = reinterpret_cast<LHCb::MDFHeader*>(&m_buffer[0]);
-    auto const skip = 4 * sizeof(int);
-    auto c = LHCb::hash32Checksum(m_buffer.data() + skip, m_buffer.size() - skip);
-    header->setChecksum(c);
-  }
-
   unsigned int mask[] = { m_partitionID, ~0x0U, ~0x0U, ~0x0U };
   void* free_address = nullptr;
   long free_len = 0;
-- 
GitLab


From a5675316ba507ab6ce07353904977eba6e805eb4 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 24 Nov 2021 14:21:18 +0100
Subject: [PATCH 054/120] Follow changes in Allen

---
 AllenOnline/src/AllenApplication.cpp | 11 +++++++----
 AllenOnline/src/AllenApplication.h   |  2 +-
 2 files changed, 8 insertions(+), 5 deletions(-)

diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 97e217f9e..8d9ee8265 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -206,12 +206,15 @@ int AllenApplication::configureApplication()
     }
   }
 
-  m_provider = dynamic_cast<IInputProvider*>(mepProvider.get());
-  if (m_provider == nullptr) {
+  auto* provider = dynamic_cast<IInputProvider*>(mepProvider.get());
+  if (provider == nullptr) {
     m_logger->error("Failed to cast MEPProvider");
     return Online::ONLINE_ERROR;
   }
 
+  // Use a deleter that does nothing to allow wrapping it in a shared pointer.
+  m_provider.reset(provider, [](IInputProvider*) {});
+
   m_json = resolveEnvVars(m_allenConfig->json);
   fs::path json{m_json};
   if (!fs::exists(json) || !fs::is_regular_file(json)) {
@@ -361,7 +364,7 @@ std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
   if (output_type == "file" || output_type == "tcp") {
     std::map<std::string, std::string> options = {{"configuration", m_json},
                                                   {"output-file", output}};
-    return Allen::output_handler(m_provider, m_zmqSvc.get(), std::move(options));
+    return Allen::output_handler(m_provider.get(), m_zmqSvc.get(), std::move(options));
   }
   else if (output_type == "mbm") {
     if (m_allenConfig->partitionBuffers.value()) {
@@ -375,7 +378,7 @@ std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
     auto output_logger = m_logger->clone("MBMOutput", m_config->outputLevel());
 
     return std::make_unique<MBMOutput>(std::move(output_logger),
-                                       m_provider,
+                                       m_provider.get(),
                                        connection,
                                        RTL::processName(),
                                        m_allenConfig->partitionID.value(),
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 2006e633b..207f3e583 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -93,7 +93,7 @@ private:
 
   std::string m_controlConnection = "inproc://AllenApplicationControl";
 
-  IInputProvider* m_provider = nullptr;
+  std::shared_ptr<IInputProvider> m_provider;
   std::unique_ptr<OutputHandler> m_output;
 
   size_t m_nSlices = 0;
-- 
GitLab


From b03e2dbe9894ba4c150b4aa9c0d2445dc86c96df Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 24 Nov 2021 14:22:44 +0100
Subject: [PATCH 055/120] Don't mask 5 most-significant bits of source ID

---
 AllenOnline/src/TransposeMEP.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 921d0a771..8412ed42d 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -228,7 +228,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       }
 
       // Store source ID, mask top 5 bits which are detector ID
-      event_offsets[2 + offset_index] = block.header->src_id & 0x7FF;
+      event_offsets[2 + offset_index] = block.header->src_id;
 
       // Initialize the first offsets using the block sizes,
       if ( offset_index < banks_count[lhcb_type] - 1 ) {
@@ -327,7 +327,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
         // Write sourceID; offset in 32bit words
         auto word_offset         = offset / sizeof( uint32_t );
-        banks_write[word_offset] = block.header->src_id & 0x7FF;
+        banks_write[word_offset] = block.header->src_id;
 
         // Write bank data
         std::memcpy( banks_write + word_offset + 1, block.payload + source_offsets[i_event], frag_size );
-- 
GitLab


From 15e74fc4c75185f29e2235883b5b21af0c542e7e Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 26 Nov 2021 12:14:15 +0100
Subject: [PATCH 056/120] WIP follow changes in Allen

---
 AllenOnline/AllenOnline/MEPProvider.h | 23 ---------------
 AllenOnline/application/read_mep.cpp  |  2 +-
 AllenOnline/options/Allen.py          |  3 +-
 AllenOnline/src/AllenApplication.cpp  | 24 ++--------------
 AllenOnline/src/AllenConfiguration.h  | 40 +++++++++++++++++++++++++--
 AllenOnline/src/MEPProvider.cpp       | 24 ++++++++--------
 6 files changed, 53 insertions(+), 63 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 01881e7c2..2528743bd 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -283,29 +283,6 @@ private:
 
   std::unordered_set<BankTypes> m_bank_types;
 
-  Gaudi::Property<std::set<LHCb::RawBank::BankType>>
-  m_lhcb_bank_types{this, "BankTypes",
-                    {LHCb::RawBank::VP,
-                     LHCb::RawBank::UT,
-                     LHCb::RawBank::FTCluster,
-                     LHCb::RawBank::Muon,
-                     LHCb::RawBank::ODIN,
-                     LHCb::RawBank::EcalPacked,
-                     LHCb::RawBank::HcalPacked},
-    [this](auto &) -> void {
-      m_bank_types.clear();
-      for (auto bt : m_lhcb_bank_types) {
-        auto it = Allen::bank_types.find(bt);
-        if (it == Allen::bank_types.end()) {
-          throw GaudiException(this->name(), "No Allen bank type defined for requested bank type "s
-                               + LHCb::RawBank::typeName(bt), StatusCode::FAILURE);
-        }
-        else {
-          m_bank_types.insert(it->second);
-        }
-      }
-    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
-
   Gaudi::Property<size_t> m_nslices {this, "NSlices", 6};
   Gaudi::Property<size_t> m_events_per_slice {this, "EventsPerSlice", 1000};
   Gaudi::Property<std::vector<std::string>> m_connections {this, "Connections"};
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index a1d2c98b7..0a5c4f4e7 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -85,7 +85,7 @@ int main(int argc, char* argv[])
     }
 
     MEP::find_blocks(mep, slice.blocks);
-    auto [sucess, banks_count, banks_version] = MEP::fill_counts(mep, bank_ids);
+    auto [sucess, banks_count, banks_version] = MEP::fill_counts(mep);
 
     if (i_mep == 0) {
       bank_slices = allocate_slices(1, bank_types,
diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 8f7d84efa..000834e40 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -82,8 +82,7 @@ allen_conf.JSON = json_file
 allen_conf.Sequence = "passthrough"
 allen_conf.NLines = n_lines
 allen_conf.OutputChecksum = False
-allen_conf.ParamDir = "/daqarea1/fest/allen/fest_geometry"
-allen_conf.Param = "/group/hlt/fest_202106/dev-dir/Allen/input/parameters"
+allen_conf.ParamDir = "/group/hlt/fest_202106/dev-dir/Allen/input/parameters"
 allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
 allen_conf.PartitionID = partition_id
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 8d9ee8265..4ac798145 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -54,19 +54,6 @@
 
 namespace {
   using namespace std::string_literals;
-
-  std::string resolveEnvVars(std::string s)
-  {
-    std::regex envExpr {"\\$\\{([A-Za-z0-9_]+)\\}"};
-    std::smatch m;
-    while (std::regex_search(s, m, envExpr)) {
-      std::string rep;
-      System::getEnv(m[1].str(), rep);
-      s = s.replace(m[1].first - 2, m[1].second + 1, rep);
-    }
-    return s;
-  }
-
   namespace fs = std::filesystem;
 } // namespace
 
@@ -215,12 +202,7 @@ int AllenApplication::configureApplication()
   // Use a deleter that does nothing to allow wrapping it in a shared pointer.
   m_provider.reset(provider, [](IInputProvider*) {});
 
-  m_json = resolveEnvVars(m_allenConfig->json);
-  fs::path json{m_json};
-  if (!fs::exists(json) || !fs::is_regular_file(json)) {
-    m_logger->error("Allen JSON configuration file does not exist or is not a regular file");
-    return Online::ONLINE_ERROR;
-  }
+  m_json = m_allenConfig->json;
 
   m_output = makeOutput();
   if (!m_allenConfig->output.value().empty() && !m_output) {
@@ -319,14 +301,12 @@ int AllenApplication::continueProcessing()
 void AllenApplication::allenLoop()
 {
   auto paramDir = resolveEnvVars(m_allenConfig->paramDir);
-  auto param = resolveEnvVars(m_allenConfig->param);
 
   //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
   //--device 23:00.0
   std::map<std::string, std::string> allen_options = {{"v", std::to_string(6 - m_config->outputLevel())},
                                                       {"t", m_allenConfig->nThreads.toString()},
-                                                      {"geometry", paramDir},
-						      {"params", param},
+                                                      {"params", paramDir},
                                                       {"configuration", m_json},
                                                       {"device", m_allenConfig->device.value()},
                                                       {"s", std::to_string(m_nSlices)},
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index e67cf1446..aa40336da 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -10,8 +10,26 @@
 \*****************************************************************************/
 #pragma once
 
+#include <filesystem>
+
 #include <GaudiKernel/Service.h>
 
+namespace {
+  std::string resolveEnvVars(std::string s)
+  {
+    std::regex envExpr {"\\$\\{([A-Za-z0-9_]+)\\}"};
+    std::smatch m;
+    while (std::regex_search(s, m, envExpr)) {
+      std::string rep;
+      System::getEnv(m[1].str(), rep);
+      s = s.replace(m[1].first - 2, m[1].second + 1, rep);
+    }
+    return s;
+  }
+
+  namespace fs = std::filesystem;
+}
+
 class AllenConfiguration : public Service {
 public:
   /// Retrieve interface ID
@@ -33,10 +51,26 @@ public:
   Gaudi::Property<std::string> output {this, "Output", ""};
   Gaudi::Property<bool> runChanges {this, "EnableRunChanges", true};
   Gaudi::Property<std::string> device {this, "Device", "0"};
-  Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json"};
+  Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json",
+    [this](auto &) -> void {
+      auto const json_file = resolveEnvVars(json.value());
+      fs::path j{json_file};
+      if (!fs::exists(j) || !fs::is_regular_file(j)) {
+        throw GaudiException{"Allen JSON configuration file does not exist or is not a regular file", name(), StatusCode::FAILURE};
+      }
+      json.set(json_file);
+    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
   Gaudi::Property<size_t> nLines{this, "NLines", 0ul};
-  Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/detector_configuration/down"};
-  Gaudi::Property<std::string> param {this, "Param", "${ALLEN_PROJECT_ROOT}/input/parameters"};
+  Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/parameters",
+    [this](auto &) -> void {
+      auto const dir = resolveEnvVars(paramDir.value());
+      fs::path p{dir};
+      if (!fs::exists(p) || !fs::is_directory(p)) {
+        throw GaudiException{"Allen parameter directory file does not exist or is not a directory", name(), StatusCode::FAILURE};
+      }
+      paramDir.set(dir);
+    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
+  }
   Gaudi::Property<std::string> sequence {this, "Sequence", "hlt1_pp_default"};
 
   Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 663136cfe..3ff23ccb5 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -4,6 +4,7 @@
 
 #include <Allen/Logger.h>
 #include <Allen/InputProvider.h>
+#include <Allen/Provider.h>
 #include <Allen/BankTypes.h>
 #include <Allen/BankMapping.h>
 #include <Allen/Timer.h>
@@ -253,16 +254,22 @@ StatusCode MEPProvider::initialize()
     m_buffer_status[i].index = i;
   }
 
-  if (m_bank_types.empty()) {
-    error() << "No bank types specified" << endmsg;
+  auto config = service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
+  if (!config) {
+    error() << "Failed to retrieve AllenConfiguration." << endmsg;
     return StatusCode::FAILURE;
   }
-  else if (m_lhcb_bank_types.size() != m_bank_types.size()) {
-    error() << "Not all LHCb bank types converted to Allen bank types" << endmsg;
+  m_allenConfig = config.get();
+
+  m_bank_types = Allen::configured_bank_types(m_allenConfig->json.value());
+  if (m_bank_types.empty()) {
+    error() << "No bank types specified" << endmsg;
     return StatusCode::FAILURE;
   }
   else {
-    info() << "Providing banks for " << m_lhcb_bank_types.toString() << endmsg;
+    info() << "Providing banks for";
+    for (auto bt : m_bank_types) info() << " " << ::bank_name(bt);
+    info() << endmsg;
   }
 
   #ifdef HAVE_MPI
@@ -309,13 +316,6 @@ StatusCode MEPProvider::initialize()
 
   m_buffer_reading = m_buffer_status.begin();
 
-  auto config = service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
-  if (!config) {
-    error() << "Failed to retrieve AllenConfiguration." << endmsg;
-    return StatusCode::FAILURE;
-  }
-  m_allenConfig = config.get();
-
   StatusCode sc = StatusCode::SUCCESS;
   if (m_source == MEP::ProviderSource::MPI) {
     sc = init_mpi();
-- 
GitLab


From ca6899435de99b0a17913032f0e10b2abfd8cf47 Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Sat, 30 Oct 2021 17:55:41 +0200
Subject: [PATCH 057/120] Identify subdetector from sourceID; count banks by
 sourceID rather than by LHCb BankType

---
 AllenOnline/AllenOnline/MEPProvider.h  |  2 +-
 AllenOnline/AllenOnline/TransposeMEP.h | 11 ++---
 AllenOnline/options/Allen.py           |  1 +
 AllenOnline/src/AllenConfiguration.h   |  1 +
 AllenOnline/src/MEPProvider.cpp        | 13 ++++--
 AllenOnline/src/TransposeMEP.cpp       | 60 ++++++++++++++++++--------
 6 files changed, 59 insertions(+), 29 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 2528743bd..9e98c936b 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -260,7 +260,7 @@ private:
   std::vector<std::thread> m_transpose_threads;
 
   // Array to store the number of banks per bank type
-  mutable std::array<unsigned int, LHCb::NBankTypes> m_banks_count;
+  mutable std::array<unsigned int, Allen::NSubDetectors> m_banks_count;
   mutable bool m_sizes_known = false;
 
   // Run and event numbers present in each slice
diff --git a/AllenOnline/AllenOnline/TransposeMEP.h b/AllenOnline/AllenOnline/TransposeMEP.h
index d36743777..28666e01e 100644
--- a/AllenOnline/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/AllenOnline/TransposeMEP.h
@@ -22,6 +22,7 @@
 #include <Allen/Common.h>
 #include <Allen/AllenUnits.h>
 #include <Allen/TransposeTypes.h>
+#include <Allen/BankMapping.h>
 #include <Event/RawBank.h>
 
 #include <EventBuilding/MFP_tools.hpp>
@@ -69,7 +70,7 @@ namespace MEP {
    *
    * @return     (success, number of banks per bank type; 0 if the bank is not needed, bank version per type)
    */
-  std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
+  std::tuple<bool, std::array<unsigned int, Allen::NSubDetectors>, std::array<int, NBankTypes>>
   fill_counts(EB::MEP const* mep, std::vector<int> const& bank_ids);
 
   void find_blocks(EB::MEP const* mep, Blocks& blocks);
@@ -81,7 +82,7 @@ namespace MEP {
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
     Blocks const& blocks,
     MEP::SourceOffsets const& input_offsets,
     std::tuple<size_t, size_t> const& interval,
@@ -92,7 +93,7 @@ namespace MEP {
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
     EventIDs& event_ids,
     MEP::Blocks const& blocks,
     std::tuple<size_t, size_t> const& interval,
@@ -114,7 +115,7 @@ namespace MEP {
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    std::array<unsigned int,Allen::NSubDetectors > const& banks_count,
     EventIDs& event_ids,
     EB::MEP const* mep,
     Blocks const& blocks,
@@ -137,7 +138,7 @@ namespace MEP {
     int const slice_index,
     std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
     EventIDs& event_ids,
     EB::MEP const* mep,
     Blocks const& blocks,
diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 000834e40..1224d8de6 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -87,6 +87,7 @@ allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
 allen_conf.PartitionID = partition_id
 allen_conf.EnableRunChanges = False
+allen_conf.Verbosity = 5
 
 if run_online:
     allen_conf.Output = "mbm://Output"
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index aa40336da..330f77dad 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -77,4 +77,5 @@ public:
   Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
   Gaudi::Property<bool> partitionBuffers {this, "PartitionBuffers", true};
   Gaudi::Property<std::string> partition {this, "Partition", ""};
+  Gaudi::Property<unsigned> verbosity {this, "Verbosity", 3};
 };
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 3ff23ccb5..363889895 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -19,6 +19,8 @@
 
 #include <MBM/Requirement.h>
 
+#include <lhcb/daq40/sourceid.h>
+
 #include "AllenOnline/MEPProvider.h"
 #include "AllenOnline/TransposeMEP.h"
 
@@ -711,9 +713,12 @@ bool MEPProvider::allocate_storage(size_t i_read)
     }
     // In case of direct MEP output, no memory should be allocated.
     if (!m_transpose_mep.value()) {
-      auto it = std::find(m_bank_ids.begin(), m_bank_ids.end(), to_integral(bank_type));
-      auto lhcb_type = std::distance(m_bank_ids.begin(), it);
-      auto n_blocks = m_banks_count[lhcb_type];
+      //auto it = std::find(m_bank_ids.begin(), m_bank_ids.end(), to_integral(bank_type));
+      //auto lhcb_type = std::distance(m_bank_ids.begin(), it);
+      auto subdetector_index = Allen::subdetector_index_from_bank_type(bank_type);
+      assert(subdetector_index > -1);
+      
+      auto n_blocks = m_banks_count[subdetector_index];
       // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
       return {0, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
     }
@@ -740,7 +745,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
       auto n_bytes = std::lround(
         ((1 + m_banks_count[lhcb_type]) * sizeof(uint32_t) + it->second * kB) * aps *
           bank_size_fudge_factor +
-        2 * MB);
+        2 * MB); // FIXME for the banks_count
       return {n_bytes, eps};
     }
   };
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 8412ed42d..33566d622 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -5,6 +5,7 @@
 #include <cstring>
 
 #include <Event/ODIN.h>
+#include <lhcb/daq40/sourceid.h>
 
 #include <AllenOnline/TransposeMEP.h>
 
@@ -20,7 +21,7 @@ namespace {
 } // namespace
 
 
-std::tuple<bool, std::array<unsigned int, LHCb::NBankTypes>, std::array<int, NBankTypes>>
+std::tuple<bool, std::array<unsigned int, Allen::NSubDetectors>, std::array<int, NBankTypes>>
 MEP::fill_counts( EB::MEP const* mep, std::vector<int> const& bank_ids ) {
   // info_cout << "EB header: "
   //   << header.n_blocks << ", "
@@ -28,7 +29,7 @@ MEP::fill_counts( EB::MEP const* mep, std::vector<int> const& bank_ids ) {
   //   << header.reserved << ", "
   //   << header.mep_size << "\n";
 
-  std::array<unsigned int, LHCb::NBankTypes> count{0};
+  std::array<unsigned int, Allen::NSubDetectors> count{0};
   std::array<int, NBankTypes>                versions{0};
   for ( size_t i = 0; i < mep->header.n_MFPs; ++i ) {
     auto const* mfp = mep->at(i);
@@ -36,8 +37,15 @@ MEP::fill_counts( EB::MEP const* mep, std::vector<int> const& bank_ids ) {
     // info_cout << "EB BlockHeader: "
     //   << bh.event_id << ", " << bh.n_frag << ", " << bh.reserved << ", " << bh.block_size << "\n";
     auto type = mfp->header.bank_types()[0];
-    if ( type < LHCb::RawBank::LastType ) { ++count[type]; }
-
+    // if ( type < LHCb::RawBank::LastType ) { ++count[type]; }
+
+    const int source_id = mfp->header.src_id;
+    const std::string subdetector = SourceId_sysstr(static_cast<SourceIdSys>(source_id));
+    auto it = Allen::subdetector_allen_bank_types.find(subdetector);
+    if (it != Allen::subdetector_allen_bank_types.end()) {
+      const auto index = std::distance(Allen::subdetector_allen_bank_types.begin(), it);
+      ++count[index];
+    }
     auto const allen_type = bank_ids[type];
     if ( allen_type != -1 ) { versions[allen_type] = mfp->header.block_version; }
   }
@@ -88,7 +96,7 @@ void MEP::fragment_offsets( MEP::Blocks const& blocks, MEP::SourceOffsets& offse
 
 size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
                            std::unordered_set<BankTypes> const&              bank_types,
-                           std::array<unsigned int, LHCb::NBankTypes> const& banks_count, MEP::Blocks const& blocks,
+                           std::array<unsigned int, Allen::NSubDetectors> const& banks_count, MEP::Blocks const& blocks,
                            MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval,
                            bool split_by_run ) {
 
@@ -167,27 +175,38 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
 std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int const slice_index,
                                                  std::vector<int> const&                           bank_ids,
                                                  std::unordered_set<BankTypes> const&              bank_types,
-                                                 std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+                                                 std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
                                                  EventIDs& event_ids, MEP::Blocks const& blocks,
                                                  std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
 
   auto [event_start, event_end] = interval;
 
-  unsigned char prev_type    = 0;
+  std::string   prev_type   = "";
   size_t        offset_index = 0;
   uint          run_number   = 0;
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
     auto const& block = blocks[i_block];
     // FIXME: BankType could be an error bank for that
     // subdetector. Use something better.
-    auto        lhcb_type  = block.bank_types[0];
-    auto        allen_type = bank_ids[lhcb_type];
-    auto        n_blocks   = banks_count[lhcb_type];
+    const int source_id = block.header->src_id;
+    //get first 5 bits and find related subdector
+    const std::string subdetector = SourceId_sysstr(static_cast<SourceIdSys>(source_id));
+    
+    info_cout << "SUBDETECTOR = " << std::string(subdetector) << std::endl;
+
+    //auto        lhcb_type  = block.bank_types[0]; //block.header.bank_types()[0];
+    //auto        lhcb_type  = block.bank_types[0]; // lhcb_type should become subdetector
+    //auto        allen_type = bank_ids[lhcb_type]; // allen_type should become subdetector
+    
+    auto allen_type = Allen::subdetector_id(subdetector);
+    const auto subdetector_index = Allen::subdetector_index(subdetector);
+
+    auto        n_blocks   = banks_count[subdetector_index]; 
     auto const* bank_sizes = block.bank_sizes;
     auto const  align      = block.header->align;
 
     // Decode ODIN banks
-    if ( lhcb_type == LHCb::RawBank::ODIN ) {
+    if ( subdetector == "ODIN" ) { // Can ODIN have an error bank?
       // decode ODIN bank to obtain run and event numbers
       unsigned fragment_offset = 0;
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
@@ -219,19 +238,19 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       for ( size_t i = event_start; i < event_end; ++i ) { interval_size += bank_sizes[i]; }
 
       // Calculate offsets
-      if ( lhcb_type != prev_type ) {
-        event_offsets[0]            = banks_count[lhcb_type];
+      if ( subdetector != prev_type ) {
+        event_offsets[0]            = banks_count[subdetector_index];
         event_offsets[1]            = event_end - event_start;
         event_offsets[2 + n_blocks] = 0;
         offset_index                = 0;
-        prev_type                   = lhcb_type;
+        prev_type                   = subdetector;
       }
 
       // Store source ID, mask top 5 bits which are detector ID
       event_offsets[2 + offset_index] = block.header->src_id;
 
       // Initialize the first offsets using the block sizes,
-      if ( offset_index < banks_count[lhcb_type] - 1 ) {
+      if ( offset_index < banks_count[subdetector_index] - 1 ) {
         event_offsets[2 + n_blocks + offset_index + 1] = event_offsets[2 + n_blocks + offset_index] + interval_size;
       }
 
@@ -257,7 +276,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
 
 bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
                            std::unordered_set<BankTypes> const&              bank_types,
-                           std::array<unsigned int, LHCb::NBankTypes> const& banks_count, EventIDs& event_ids,
+                           std::array<unsigned int, Allen::NSubDetectors> const& banks_count, EventIDs& event_ids,
                            EB::MEP const* mep, MEP::Blocks const& blocks,
                            MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval ) {
   auto [start_event, end_event] = interval;
@@ -272,6 +291,9 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
     auto        bank_type      = static_cast<LHCb::RawBank::BankType>( block.bank_types[0] );
     auto&       source_offsets = input_offsets[i_block];
     auto const* bank_sizes     = block.bank_sizes;
+    const int source_id = block.header->src_id;
+    const std::string subdetector = SourceId_sysstr(static_cast<SourceIdSys>(source_id));
+    const auto subdetector_index = Allen::subdetector_index(subdetector);
 
     // Check what to do with this bank
     if ( bank_type == LHCb::RawBank::ODIN ) {
@@ -302,7 +324,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
         // - offsets to individual banks
         // - bank data
 
-        auto preamble_words = 2 + banks_count[bank_type];
+        auto preamble_words = 2 + banks_count[subdetector_index];
 
         // Initialize point to write from offset of previous set
         // All bank offsets are uit32_t so cast to that type
@@ -313,7 +335,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
         if ( bank_index == 1 ) {
           // Write the number of banks
-          banks_write[0]         = banks_count[bank_type];
+          banks_write[0]         = banks_count[subdetector_index];
           banks_offsets_write[0] = 0;
         }
 
@@ -341,7 +363,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
 std::tuple<bool, bool, size_t> MEP::transpose_events(
     Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
-    std::unordered_set<BankTypes> const& bank_types, std::array<unsigned int, LHCb::NBankTypes> const& banks_count,
+    std::unordered_set<BankTypes> const& bank_types, std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
     EventIDs& event_ids, EB::MEP const* mep, MEP::Blocks const& blocks,
     MEP::SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
   auto [event_start, event_end] = interval;
-- 
GitLab


From 87c5e3578e528d5ccdec17c31093f5091c7dfccc Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 11 Nov 2021 15:54:43 +0100
Subject: [PATCH 058/120] Use source IDs instead of bank types to identify MEPs

---
 AllenOnline/AllenOnline/MEPProvider.h  |   6 +-
 AllenOnline/AllenOnline/TransposeMEP.h |  21 +--
 AllenOnline/CMakeLists.txt             | 121 ++++++------
 AllenOnline/application/read_mep.cpp   |  27 +--
 AllenOnline/src/MEPProvider.cpp        |  43 ++---
 AllenOnline/src/TransposeMEP.cpp       | 244 ++++++++++++-------------
 cmake/MooreOnlineDependencies.cmake    |   2 +
 7 files changed, 228 insertions(+), 236 deletions(-)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 9e98c936b..35489ddd3 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -236,9 +236,6 @@ private:
   mutable std::atomic<bool> m_read_error = false;
   std::atomic<bool> m_transpose_done = false;
 
-  // Allen IDs of LHCb raw banks
-  std::vector<int> m_bank_ids;
-
   // Memory slices, N for each raw bank type
   Allen::Slices m_slices;
   std::vector<std::tuple<int, size_t, size_t>> m_slice_to_buffer;
@@ -260,11 +257,12 @@ private:
   std::vector<std::thread> m_transpose_threads;
 
   // Array to store the number of banks per bank type
-  mutable std::array<unsigned int, Allen::NSubDetectors> m_banks_count;
+  mutable std::array<unsigned int, NBankTypes> m_mfp_count;
   mutable bool m_sizes_known = false;
 
   // Run and event numbers present in each slice
   std::vector<EventIDs> m_event_ids;
+  std::vector<std::vector<char>> m_event_masks;
 
   // Storage for the currently open input file
   mutable std::optional<LHCb::StreamDescriptor::Access> m_input = std::nullopt;
diff --git a/AllenOnline/AllenOnline/TransposeMEP.h b/AllenOnline/AllenOnline/TransposeMEP.h
index 28666e01e..0c6cd0d22 100644
--- a/AllenOnline/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/AllenOnline/TransposeMEP.h
@@ -70,8 +70,8 @@ namespace MEP {
    *
    * @return     (success, number of banks per bank type; 0 if the bank is not needed, bank version per type)
    */
-  std::tuple<bool, std::array<unsigned int, Allen::NSubDetectors>, std::array<int, NBankTypes>>
-  fill_counts(EB::MEP const* mep, std::vector<int> const& bank_ids);
+  std::tuple<bool, std::array<unsigned int, NBankTypes>, std::array<int, NBankTypes>>
+  fill_counts(EB::MEP const* mep);
 
   void find_blocks(EB::MEP const* mep, Blocks& blocks);
 
@@ -80,9 +80,8 @@ namespace MEP {
   size_t allen_offsets(
     Allen::Slices& slices,
     int const slice_index,
-    std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
+    std::array<unsigned int, NBankTypes> const& mfp_count,
     Blocks const& blocks,
     MEP::SourceOffsets const& input_offsets,
     std::tuple<size_t, size_t> const& interval,
@@ -91,10 +90,10 @@ namespace MEP {
   std::tuple<bool, bool, size_t> mep_offsets(
     Allen::Slices& slices,
     int const slice_index,
-    std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
+    std::array<unsigned int, NBankTypes> const& banks_count,
     EventIDs& event_ids,
+    std::vector<char>& event_mask,
     MEP::Blocks const& blocks,
     std::tuple<size_t, size_t> const& interval,
     bool split_by_run = false);
@@ -110,13 +109,13 @@ namespace MEP {
    *
    * @return     tuple of: (success, slice is full)
    */
-  bool transpose_event(
+  bool transpose_mep(
     Allen::Slices& slices,
     int const slice_index,
-    std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int,Allen::NSubDetectors > const& banks_count,
+    std::array<unsigned int, NBankTypes > const& mfp_count,
     EventIDs& event_ids,
+    std::vector<char>& event_mask,
     EB::MEP const* mep,
     Blocks const& blocks,
     SourceOffsets const& input_offsets,
@@ -136,10 +135,10 @@ namespace MEP {
   std::tuple<bool, bool, size_t> transpose_events(
     Allen::Slices& slices,
     int const slice_index,
-    std::vector<int> const& bank_ids,
     std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
+    std::array<unsigned int, NBankTypes> const& mfp_count,
     EventIDs& event_ids,
+    std::vector<char>& event_mask,
     EB::MEP const* mep,
     Blocks const& blocks,
     SourceOffsets const& source_offsets,
diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 0195ee4c8..7b27143b3 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -8,52 +8,50 @@
 # granted to it by virtue of its status as an Intergovernmental Organization  #
 # or submit itself to any jurisdiction.                                       #
 ###############################################################################
-################################################################################
-# Package: AllenOnline
-################################################################################
-gaudi_subdir(AllenOnline v1r0)
+#[=======================================================================[.rst:
+AllenOnline
+-------------
+#]=======================================================================]
 
-gaudi_depends_on_subdirs(Online/OnlineKernel
-                         Online/GaudiOnline
-                         Online/dim
-                         Online/RPC
-                         Tools/ZeroMQ
-                         Online/OnlineBase
-                         Online/Parsers
-                         Online/EventBuilding
-                         Dumpers/BinaryDumpers)
-
-find_package(cppgsl REQUIRED)
-find_package(ZMQ REQUIRED)
-find_package(Boost REQUIRED COMPONENTS program_options)
-
-include_directories(SYSTEM ${Boost_INCLUDE_DIRS}
-  ${ZMQ_INCLUDE_DIRS}
-  ${CPPGSL_INCLUDE_DIR})
-
-include_directories(include)
-
-gaudi_add_library(AllenOnline
-                 src/ReadMEP.cpp
-                 src/TransposeMEP.cpp
-                 PUBLIC_HEADERS AllenOnline
-                 INCLUDE_DIRS GaudiKernel cppgsl Online/OnlineKernel Online/EventBuilding
-                 LINK_LIBRARIES GaudiKernel Allen::AllenLib EventBuildingLib)
+gaudi_add_library(AllenOnlineLib
+    SOURCES
+        src/ReadMEP.cpp
+        src/TransposeMEP.cpp
+    LINK PUBLIC
+        Gaudi::GaudiKernel
+        cppgsl::cppgsl
+        Online::OnlineKernel
+        Online::EventBuildingLib
+        Allen::AllenLib
+)
 
 
 gaudi_add_module(AllenOnlineComp
-                 src/AllenApplication.cpp
-                 src/AllenConfiguration.cpp
-                 src/MEPProvider.cpp
-                 src/MBMOutput.cpp
-                 INCLUDE_DIRS Online/OnlineBase Online/DIM GaudiKernel cppgsl Online/OnlineKernel Tools/ZeroMQ Online/EventBuilding
-                 LINK_LIBRARIES GaudiKernel dim OnlineBase GaudiOnline ZMQLib RPC Parsers EventBuildingLib MDFLib
-                                OnlineKernel Allen::AllenLib BinaryDumpers DAQEventLib
-                                DAQKernelLib GaudiAlgLib PrKernel VPDetLib UTDetLib UTKernelLib AllenOnline
-                                FTDetLib -lrt)
+    SOURCES
+        src/AllenApplication.cpp
+        src/AllenConfiguration.cpp
+        src/MEPProvider.cpp
+        src/MBMOutput.cpp
+    LINK
+        Gaudi::GaudiKernel
+        Online::dim
+        Online::OnlineBase
+        Online::GaudiOnline
+        LHCb::ZMQLib
+        Online::RPC
+        Gaudi::Parsers
+        Online::EventBuildingLib
+        LHCb::MDFLib
+        Online::OnlineKernel
+        Allen::AllenLib
+        Allen::BinaryDumpers
+        LHCb::DAQEventLib
+        LHCb::DAQKernelLib
+        Gaudi::GaudiAlgLib
+        AllenOnlineLib
+        rt)
 
 # MPI
-find_package(MPI QUIET COMPONENTS C CXX)
 if (MPI_FOUND)
   message(STATUS "Found MPI ${MPI_CXX_VERSION}: ${MPI_CXX_COMPILER}")
   #hwloc is also needed
@@ -72,23 +70,32 @@ if (MPI_FOUND)
   target_include_directories(AllenOnlineComp PRIVATE SYSTEM ${MPI_CXX_INCLUDE_DIRS})
   target_compile_definitions(AllenOnlineComp PRIVATE HAVE_MPI)
 
-
-  gaudi_add_executable(AllenMPISend
-                       application/MPISend.cpp
-                       INCLUDE_DIRS GaudiKernel ${MPI_CXX_INCLUDE_DIRS}
-                       LINK_LIBRARIES AllenOnline MDFLib Boost::program_options MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
-  target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
-
-else()
-  message(STATUS "Could not find MPI")
+  gaudi_add_executable(allen_mpi_send
+      SOURCES
+        application/MPISend.cpp
+      LINK
+        AllenOnline LHCb::MDFLib
+        Boost::program_options
+        MPI::MPI_CXX PkgConfig::hwloc
+        ${open_pal_lib})
+   target_include_directories(allen_mpi_send PRIVATE ${MPI_CXX_INCLUDE_DIRS})
+   target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
 endif()
 
-gaudi_add_executable(test_read_mep
-                     application/read_mep.cpp
-                     INCLUDE_DIRS GaudiKernel
-                     LINK_LIBRARIES AllenOnline MDFLib Boost::program_options)
+gaudi_add_executable(allen_read_mep
+    SOURCES
+      application/read_mep.cpp
+    LINK
+      AllenOnline
+      LHCb::MDFLib
+      Boost::program_options)
 
-gaudi_add_executable(test_host_register
-                     application/test_host_register.cpp
-                     INCLUDE_DIRS OnlineBase/OnlineBase
-                     LINK_LIBRARIES GaudiKernel OnlineBase Allen::Backend ZMQLib)
+gaudi_add_executable(allen_test_host_register
+    SOURCES
+      application/test_host_register.cpp
+    LINK
+      Gaudi::GaudiKernel
+      Online::OnlineBase
+      Allen::Backend
+      LHCb::ZMQLib
+    TEST)
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 0a5c4f4e7..697cf563a 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -19,6 +19,7 @@
 
 #include <Allen/Logger.h>
 #include <Allen/SliceUtils.h>
+#include <Allen/sourceid.h>
 
 #include <MDF/StreamDescriptor.h>
 
@@ -55,7 +56,7 @@ int main(int argc, char* argv[])
   vector<char> data;
 
   EventIDs event_ids;
-  auto bank_ids = Allen::bank_ids();
+  std::vector<char> event_mask;
 
   MEP::Slices mep_slices(1);
   auto& slice = mep_slices[0];
@@ -77,6 +78,7 @@ int main(int argc, char* argv[])
 
     if (i_mep == 0) {
       event_ids.reserve(slice.packing_factor);
+      event_mask.resize(slice.packing_factor, 0);
       slice.blocks.resize(mep->header.n_MFPs, MEP::Blocks::value_type{});
       slice.offsets.resize(mep->header.n_MFPs);
       for (auto& offsets : slice.offsets) {
@@ -85,14 +87,12 @@ int main(int argc, char* argv[])
     }
 
     MEP::find_blocks(mep, slice.blocks);
-    auto [sucess, banks_count, banks_version] = MEP::fill_counts(mep);
+    auto [success, mfp_count, banks_version] = MEP::fill_counts(mep);
 
     if (i_mep == 0) {
       bank_slices = allocate_slices(1, bank_types,
-        [pf = slice.packing_factor, &bank_ids, bc = banks_count](auto bt) -> std::tuple<size_t, size_t> {
-          auto it = std::find(bank_ids.begin(), bank_ids.end(), to_integral(bt));
-          auto lhcb_type = std::distance(bank_ids.begin(), it);
-          auto n_blocks = bc[lhcb_type];
+        [pf = slice.packing_factor, &mc = mfp_count](auto bt) -> std::tuple<size_t, size_t> {
+          auto n_blocks = mc[to_integral(bt)];
           return {0ul, 2 + n_blocks + (1 + pf) * (1 + n_blocks) - 2};
         });
     }
@@ -102,7 +102,9 @@ int main(int argc, char* argv[])
 
     for (auto const& block : slice.blocks) {
       // block offsets are in number of 4-byte words
-      auto lhcb_type = int {block.bank_types[0]};
+      auto const source_id = block.header->src_id;
+      std::string det = SourceId_sysstr(source_id);
+      std::string fill(7 - det.size(), ' ');
 
       size_t padded_size = 0;
       auto const* bank_sizes = block.bank_sizes;
@@ -110,20 +112,23 @@ int main(int argc, char* argv[])
         padded_size += bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << block.header->align);
       }
 
+
       cout << "fragment"
            << " magic: 0x" << std::hex << block.header->magic << std::dec
            << " packing: " << std::setw(4) << block.header->n_banks << " event_id: " << std::setw(6)
-           << block.header->ev_id << " type: " << std::setw(3) << lhcb_type << " source_id " << std::setw(4)
-           << (block.header->src_id & 0x7FF) << " version: " << std::setw(2) << unsigned{block.header->block_version}
+           << block.header->ev_id << " source_id: " << std::setw(6) << source_id
+           << " top5: " << std::setw(2) << SourceId_sys(source_id) << fill << " (" << det << ") "
+           << std::setw(5) << SourceId_num(source_id)
+           << " version: " << std::setw(2) << unsigned{block.header->block_version}
            << " size: " << std::setw(8) << block.header->bytes() - block.header->header_size()
            << " padded_size: " << std::setw(8) << padded_size << "\n";
     }
 
     MEP::mep_offsets(bank_slices, 0,
-                     bank_ids,
                      bank_types,
-                     banks_count,
+                     mfp_count,
                      event_ids,
+                     event_mask,
                      slice.blocks,
                      {0ul, slice.packing_factor},
                      false);
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 363889895..3fd2b95a9 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -10,6 +10,7 @@
 #include <Allen/Timer.h>
 #include <Allen/SliceUtils.h>
 #include <Allen/write_mdf.hpp>
+#include <Allen/sourceid.h>
 #include <Backend/BackendCommon.h>
 
 #include <MDF/StreamDescriptor.h>
@@ -19,7 +20,6 @@
 
 #include <MBM/Requirement.h>
 
-#include <lhcb/daq40/sourceid.h>
 
 #include "AllenOnline/MEPProvider.h"
 #include "AllenOnline/TransposeMEP.h"
@@ -55,7 +55,7 @@ DECLARE_COMPONENT(MEPProvider)
 
 MEPProvider::MEPProvider(std::string name, ISvcLocator* loc)
   : Service {name, loc},
-    m_banks_count {0}
+    m_mfp_count {0}
 {
 }
 
@@ -250,6 +250,7 @@ StatusCode MEPProvider::initialize()
 {
   m_slice_free.resize(m_nslices.value(), true);
   m_event_ids.resize(m_nslices.value());
+  m_event_masks.resize(m_nslices.value());
 
   m_buffer_status.resize(n_buffers());
   for (size_t i = 0; i <  m_buffer_status.size(); ++i) {
@@ -335,9 +336,6 @@ StatusCode MEPProvider::initialize()
     m_event_ids[n].reserve(m_events_per_slice);
   }
 
-  // Cached bank LHCb bank type to Allen bank type mapping
-  m_bank_ids = Allen::bank_ids();
-
   return sc;
 }
 
@@ -683,6 +681,10 @@ bool MEPProvider::allocate_storage(size_t i_read)
     assert(slice.packing_factor == m_packing_factor);
   }
 
+  for (auto& mask : m_event_masks) {
+    mask.resize(m_packing_factor, 0);
+  }
+
   size_t const eps = m_events_per_slice.value();
   auto n_interval = m_packing_factor / eps;
   auto rest = m_packing_factor % eps;
@@ -701,38 +703,26 @@ bool MEPProvider::allocate_storage(size_t i_read)
     }
   }
 
-  std::tie(count_success, m_banks_count, m_banks_version) = MEP::fill_counts(mep, m_bank_ids);
+  std::tie(count_success, m_mfp_count, m_banks_version) = MEP::fill_counts(mep);
 
   // Allocate slice memory that will contain transposed banks ready
   // for processing by the Allen kernels
   auto size_fun = [this, eps](BankTypes bank_type) -> std::tuple<size_t, size_t> {
     auto it = BankSizes.find(bank_type);
-    auto ib = to_integral(bank_type);
+    auto const sd_index = to_integral(bank_type);
+
     if (it == end(BankSizes)) {
-      throw std::out_of_range {std::string {"Bank type "} + std::to_string(ib) + " has no known size"};
+      throw std::out_of_range {std::string {"Bank type "} + bank_name(bank_type) + " has no known size"};
     }
     // In case of direct MEP output, no memory should be allocated.
     if (!m_transpose_mep.value()) {
-      //auto it = std::find(m_bank_ids.begin(), m_bank_ids.end(), to_integral(bank_type));
-      //auto lhcb_type = std::distance(m_bank_ids.begin(), it);
-      auto subdetector_index = Allen::subdetector_index_from_bank_type(bank_type);
-      assert(subdetector_index > -1);
-      
-      auto n_blocks = m_banks_count[subdetector_index];
+      auto n_blocks = m_mfp_count[sd_index];
       // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
       return {0, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
     }
     else {
       auto aps = eps < 100 ? 100 : eps;
       // Lookup LHCb bank type corresponding to Allen bank type
-      auto type_it =
-        std::find_if(Allen::bank_types.begin(), Allen::bank_types.end(), [bank_type](const auto& entry) {
-          return entry.second == bank_type;
-        });
-      if (type_it == Allen::bank_types.end()) {
-        throw std::out_of_range {std::string {"Failed to lookup LHCb type for bank type "} + std::to_string(ib)};
-      }
-      auto lhcb_type = to_integral<LHCb::RawBank::BankType>(type_it->first);
 
       // When events are transposed from the read buffer into
       // the per-rawbank-type slices, a check is made each time
@@ -743,7 +733,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
       // problems for banks with very low average size like the
       // ODIN bank - 0.1 kB, a fixed amount is also added.
       auto n_bytes = std::lround(
-        ((1 + m_banks_count[lhcb_type]) * sizeof(uint32_t) + it->second * kB) * aps *
+        ((1 + m_mfp_count[sd_index]) * sizeof(uint32_t) + it->second * kB) * aps *
           bank_size_fudge_factor +
         2 * MB); // FIXME for the banks_count
       return {n_bytes, eps};
@@ -1475,6 +1465,7 @@ void MEPProvider::transpose(int thread_id)
 
     // Reset the slice
     auto& event_ids = m_event_ids[*slice_index];
+    auto& event_mask = m_event_masks[*slice_index];
     reset_slice(m_slices, *slice_index, m_bank_types, event_ids, !m_transpose_mep.value());
 
     // MEP data
@@ -1492,10 +1483,10 @@ void MEPProvider::transpose(int thread_id)
       std::tie(good, transpose_full, n_transposed) = MEP::transpose_events(
         m_slices,
         *slice_index,
-        m_bank_ids,
         m_bank_types,
-        m_banks_count,
+        m_mfp_count,
         event_ids,
+        event_mask,
         slice.mep,
         slice.blocks,
         slice.offsets,
@@ -1510,7 +1501,7 @@ void MEPProvider::transpose(int thread_id)
     else {
       // Calculate fragment offsets in MEP per sub-detector
       std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
-        m_slices, *slice_index, m_bank_ids, m_bank_types, m_banks_count, event_ids, slice.blocks, interval);
+        m_slices, *slice_index, m_bank_types, m_mfp_count, event_ids, event_mask, slice.blocks, interval);
       if (msgLevel(MSG::DEBUG)) {
         debug() << "Transpose " << thread_id << ": Calculated MEP offsets for slice " << *slice_index
                 << "; good: " << good << "; full: " << transpose_full
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 33566d622..b5af75cf7 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -5,7 +5,7 @@
 #include <cstring>
 
 #include <Event/ODIN.h>
-#include <lhcb/daq40/sourceid.h>
+#include <Allen/sourceid.h>
 
 #include <AllenOnline/TransposeMEP.h>
 
@@ -18,36 +18,29 @@ namespace {
       return LHCb::ODIN{odin_span};
     }
   }
+
+  BankTypes source_id_type(uint16_t src_id) {
+    auto const sd = SourceId_sys(src_id);
+    auto it = Allen::subdetectors.find(static_cast<SourceIdSys>(sd));
+    return it == Allen::subdetectors.end() ? BankTypes::Unknown : it->second;
+  }
 } // namespace
 
 
-std::tuple<bool, std::array<unsigned int, Allen::NSubDetectors>, std::array<int, NBankTypes>>
-MEP::fill_counts( EB::MEP const* mep, std::vector<int> const& bank_ids ) {
-  // info_cout << "EB header: "
-  //   << header.n_blocks << ", "
-  //   << header.packing_factor << ", "
-  //   << header.reserved << ", "
-  //   << header.mep_size << "\n";
+std::tuple<bool, std::array<unsigned int, NBankTypes>, std::array<int, NBankTypes>>
+MEP::fill_counts( EB::MEP const* mep ) {
 
-  std::array<unsigned int, Allen::NSubDetectors> count{0};
-  std::array<int, NBankTypes>                versions{0};
+  std::array<unsigned, NBankTypes> count{0};
+  std::array<int, NBankTypes> versions{0};
   for ( size_t i = 0; i < mep->header.n_MFPs; ++i ) {
     auto const* mfp = mep->at(i);
-
-    // info_cout << "EB BlockHeader: "
-    //   << bh.event_id << ", " << bh.n_frag << ", " << bh.reserved << ", " << bh.block_size << "\n";
-    auto type = mfp->header.bank_types()[0];
-    // if ( type < LHCb::RawBank::LastType ) { ++count[type]; }
-
     const int source_id = mfp->header.src_id;
-    const std::string subdetector = SourceId_sysstr(static_cast<SourceIdSys>(source_id));
-    auto it = Allen::subdetector_allen_bank_types.find(subdetector);
-    if (it != Allen::subdetector_allen_bank_types.end()) {
-      const auto index = std::distance(Allen::subdetector_allen_bank_types.begin(), it);
-      ++count[index];
+    auto const allen_type = source_id_type(source_id);
+    if (allen_type != BankTypes::Unknown) {
+      auto const sd_index = to_integral(allen_type);
+      versions[sd_index] = mfp->header.block_version;
+      ++count[sd_index];
     }
-    auto const allen_type = bank_ids[type];
-    if ( allen_type != -1 ) { versions[allen_type] = mfp->header.block_version; }
   }
 
   return {true, count, versions};
@@ -65,10 +58,7 @@ void MEP::find_blocks( EB::MEP const* mep, Blocks& blocks ) {
   std::vector<size_t> perm( tmp.size() );
   std::iota( perm.begin(), perm.end(), 0U );
   std::sort( perm.begin(), perm.end(), [&tmp, src_ids]( size_t a, size_t b ) {
-    if ( src_ids[a] == src_ids[b] ) {
-      return tmp[a].bank_types[0] < tmp[b].bank_types[0];
-    } else {
-      return src_ids[a] < src_ids[b];
+    return src_ids[a] < src_ids[b];
     }
   } );
 
@@ -94,9 +84,9 @@ void MEP::fragment_offsets( MEP::Blocks const& blocks, MEP::SourceOffsets& offse
   }
 }
 
-size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
-                           std::unordered_set<BankTypes> const&              bank_types,
-                           std::array<unsigned int, Allen::NSubDetectors> const& banks_count, MEP::Blocks const& blocks,
+size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
+                           std::unordered_set<BankTypes> const& bank_types,
+                           std::array<unsigned int, NBankTypes> const& mfp_count, MEP::Blocks const& blocks,
                            MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval,
                            bool split_by_run ) {
 
@@ -104,21 +94,28 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
 
   // Loop over all bank sizes in all blocks
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
+    auto&       source_offsets = input_offsets[i_block];
+
     auto const& block = blocks[i_block];
     auto const* bank_sizes     = block.bank_sizes;
     auto const  align          = block.header->align;
-    auto        lhcb_type      = block.bank_types[0];
-    auto        allen_type     = bank_ids[lhcb_type];
-    auto&       source_offsets = input_offsets[i_block];
+    auto const  allen_type     = source_id_type(block.header->src_id);
+    auto const  sd_index       = to_integral(allen_type);
     unsigned    run_number     = 0;
-    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
+
+    if ( bank_types.count( allen_type ) ) {
+      auto& event_offsets = std::get<2>( slices[sd_index][slice_index] );
+
       for ( size_t i = event_start; i < event_end; ++i ) {
         // First check for run changes in ODIN banks
-        if ( split_by_run && lhcb_type == LHCb::RawBank::ODIN ) {
+        if ( split_by_run && allen_type == BankTypes::ODIN ) {
           // decode ODIN banks to check for run changes
+          if (block.bank_types[i] != LHCb::RawBank::ODIN) continue;
+
           auto const odin = decode_odin(block.payload, source_offsets[i],
                                         bank_sizes[i], block.header->block_version);
           unsigned const odin_run_number = odin.runNumber();
+
           // if splitting by run, check all events have same run number
           if ( i == event_start ) {
             run_number = odin_run_number;
@@ -127,10 +124,9 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
             break;
           }
         }
+
         // Anticipate offset structure already here, i.e. don't assign to the first one
         auto  idx           = i - event_start + 1;
-        auto& event_offsets = std::get<2>( slices[allen_type][slice_index] );
-
         // Allen raw bank format has the sourceID followed by the raw bank data
         event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
       }
@@ -140,97 +136,87 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index, std::ve
   // Prefix sum over sizes per bank type per event to get the output
   // "Allen" offsets per bank type per event
   size_t n_frag = ( event_end - event_start );
-  for ( size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type ) {
-    auto allen_type = bank_ids[lhcb_type];
-    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
-      auto& [slice, slice_size, event_offsets, offsets_size] = slices[allen_type][slice_index];
-      event_offsets[0]                                       = 0;
-      auto preamble_words                                    = 2 + banks_count[lhcb_type];
-      for ( size_t i = 1; i <= ( event_end - event_start ) && i <= n_frag; ++i ) {
-
-        // Allen raw bank format has the number of banks and the bank
-        // offsets in a preamble
-        event_offsets[i] += preamble_words * sizeof( uint32_t ) + event_offsets[i - 1];
-
-        // Check for sufficient space
-        if ( event_offsets[i] > slice_size ) {
-          n_frag = i - 1;
-          break;
-        }
+  for ( auto allen_type : bank_types ) {
+    auto const sd_index = to_integral(allen_type);
+    auto& [slice, slice_size, event_offsets, offsets_size] = slices[sd_index][slice_index];
+    event_offsets[0]                                       = 0;
+    auto preamble_words                                    = 2 + mfp_count[sd_index];
+    for ( size_t i = 1; i <= ( event_end - event_start ) && i <= n_frag; ++i ) {
+
+      // Allen raw bank format has the number of banks and the bank
+      // offsets in a preamble
+      event_offsets[i] += preamble_words * sizeof( uint32_t ) + event_offsets[i - 1];
+
+      // Check for sufficient space
+      if ( event_offsets[i] > slice_size ) {
+        n_frag = i - 1;
+        break;
       }
     }
   }
 
   // Set offsets_size here to make sure it's consistent with the max
-  for ( size_t lhcb_type = 0; lhcb_type < bank_ids.size(); ++lhcb_type ) {
-    auto allen_type = bank_ids[lhcb_type];
-    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
-      auto& offsets_size = std::get<3>( slices[allen_type][slice_index] );
-      offsets_size       = n_frag + 1;
-    }
+  for ( auto allen_type : bank_types ) {
+    auto const sd_index = to_integral(allen_type);
+    auto& offsets_size = std::get<3>( slices[sd_index][slice_index] );
+    offsets_size       = n_frag + 1;
   }
   return n_frag;
 }
 
 std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int const slice_index,
-                                                 std::vector<int> const&                           bank_ids,
-                                                 std::unordered_set<BankTypes> const&              bank_types,
-                                                 std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
-                                                 EventIDs& event_ids, MEP::Blocks const& blocks,
+                                                 std::unordered_set<BankTypes> const& bank_types,
+                                                 std::array<unsigned int, NBankTypes> const& mfp_count,
+                                                 EventIDs& event_ids, std::vector<char>& event_mask,
+                                                 MEP::Blocks const& blocks,
                                                  std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
 
   auto [event_start, event_end] = interval;
 
-  std::string   prev_type   = "";
+  BankTypes     prev_type   = BankTypes::Unknown;
   size_t        offset_index = 0;
-  uint          run_number   = 0;
+  std::optional<unsigned> run_number = std::nullopt;
+
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const& block = blocks[i_block];
-    // FIXME: BankType could be an error bank for that
-    // subdetector. Use something better.
-    const int source_id = block.header->src_id;
-    //get first 5 bits and find related subdector
-    const std::string subdetector = SourceId_sysstr(static_cast<SourceIdSys>(source_id));
-    
-    info_cout << "SUBDETECTOR = " << std::string(subdetector) << std::endl;
-
-    //auto        lhcb_type  = block.bank_types[0]; //block.header.bank_types()[0];
-    //auto        lhcb_type  = block.bank_types[0]; // lhcb_type should become subdetector
-    //auto        allen_type = bank_ids[lhcb_type]; // allen_type should become subdetector
-    
-    auto allen_type = Allen::subdetector_id(subdetector);
-    const auto subdetector_index = Allen::subdetector_index(subdetector);
-
-    auto        n_blocks   = banks_count[subdetector_index]; 
+    auto const& block      = blocks[i_block];
+    auto const  source_id  = block.header->src_id;
+    auto const  allen_type = source_id_type(source_id);
+    auto const  sd_index   = to_integral(allen_type);
+    auto        n_blocks   = mfp_count[sd_index];
     auto const* bank_sizes = block.bank_sizes;
     auto const  align      = block.header->align;
 
     // Decode ODIN banks
-    if ( subdetector == "ODIN" ) { // Can ODIN have an error bank?
+    if ( allen_type == BankTypes::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
       unsigned fragment_offset = 0;
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
-        if ( i_event >= event_start ) {
+        if ( i_event >= event_start && block.bank_types[i_event] == LHCb::RawBank::ODIN) {
           auto const odin = decode_odin(block.payload, fragment_offset,
                                         bank_sizes[i_event], block.header->block_version);
           unsigned const odin_run_number = odin.runNumber();
           auto const event_number = odin.eventNumber();
 
           // if splitting by run, check all events have same run number
-          if ( i_event == event_start ) {
+          if ( !run_number ) {
             run_number = odin_run_number;
-          } else if ( split_by_run && odin_run_number != run_number ) {
+          } else if ( split_by_run && run_number && odin_run_number != *run_number ) {
             event_end = i_event;
             break;
           }
           event_ids.emplace_back( odin_run_number, event_number );
+          event_mask[i_event - event_start] = 1;
+        }
+        else {
+          event_ids.emplace_back( 0, 0 );
+          event_mask[i_event - event_start] = 0;
         }
         fragment_offset += bank_sizes[i_event] + EB::get_padding(bank_sizes[i_event], 1 << align);
       }
     }
 
-    if ( allen_type != -1 && bank_types.count( BankTypes{allen_type} ) ) {
-      auto& [spans, data_size, event_offsets, offsets_size] = slices[allen_type][slice_index];
+    if ( bank_types.count( allen_type ) ) {
+      auto& [spans, data_size, event_offsets, offsets_size] = slices[sd_index][slice_index];
 
       // Calculate block offset and size
       size_t interval_offset = 0, interval_size = 0;
@@ -238,19 +224,19 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       for ( size_t i = event_start; i < event_end; ++i ) { interval_size += bank_sizes[i]; }
 
       // Calculate offsets
-      if ( subdetector != prev_type ) {
-        event_offsets[0]            = banks_count[subdetector_index];
+      if ( allen_type != prev_type ) {
+        event_offsets[0]            = mfp_count[sd_index];
         event_offsets[1]            = event_end - event_start;
         event_offsets[2 + n_blocks] = 0;
         offset_index                = 0;
-        prev_type                   = subdetector;
+        prev_type                   = allen_type;
       }
 
-      // Store source ID, mask top 5 bits which are detector ID
-      event_offsets[2 + offset_index] = block.header->src_id;
+      // Store source ID
+      event_offsets[2 + offset_index] = source_id;
 
       // Initialize the first offsets using the block sizes,
-      if ( offset_index < banks_count[subdetector_index] - 1 ) {
+      if ( offset_index < mfp_count[sd_index] - 1 ) {
         event_offsets[2 + n_blocks + offset_index + 1] = event_offsets[2 + n_blocks + offset_index] + interval_size;
       }
 
@@ -274,49 +260,53 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
   return {true, false, event_end - event_start};
 }
 
-bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
-                           std::unordered_set<BankTypes> const&              bank_types,
-                           std::array<unsigned int, Allen::NSubDetectors> const& banks_count, EventIDs& event_ids,
-                           EB::MEP const* mep, MEP::Blocks const& blocks,
-                           MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval ) {
+bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
+                         std::unordered_set<BankTypes> const& bank_types,
+                         std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
+                         std::vector<char>& event_mask,
+                         EB::MEP const* mep, MEP::Blocks const& blocks,
+                         MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval ) {
   auto [start_event, end_event] = interval;
 
   // Loop over all bank data of this event
   size_t bank_index = 1;
-  // L0Calo doesn't exist in the upgrade
-  LHCb::RawBank::BankType prev_type = LHCb::RawBank::L0Calo;
+  auto prev_type = BankTypes::Unknown;
 
   for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) {
     auto const& block          = blocks[i_block];
-    auto        bank_type      = static_cast<LHCb::RawBank::BankType>( block.bank_types[0] );
+    auto const  source_id      = block.header->src_id;
+    auto        allen_type     = source_id_type(source_id);
     auto&       source_offsets = input_offsets[i_block];
     auto const* bank_sizes     = block.bank_sizes;
-    const int source_id = block.header->src_id;
-    const std::string subdetector = SourceId_sysstr(static_cast<SourceIdSys>(source_id));
-    const auto subdetector_index = Allen::subdetector_index(subdetector);
 
     // Check what to do with this bank
-    if ( bank_type == LHCb::RawBank::ODIN ) {
+    if ( allen_type == BankTypes::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
       for ( uint16_t i_event = start_event; i_event < end_event; ++i_event ) {
-        auto const odin = decode_odin(block.payload, source_offsets[i_event],
-                                      (bank_sizes[i_event] / sizeof(std::uint32_t)), block.header->block_version);
-        event_ids.emplace_back( odin.runNumber(), odin.eventNumber() );
+        if (block.bank_types[i_event] != LHCb::RawBank::ODIN) {
+          event_ids.emplace_back( 0, 0 );
+          event_mask[i_event] = 0;
+        }
+        else {
+          auto const odin = decode_odin(block.payload, source_offsets[i_event],
+                                        (bank_sizes[i_event] / sizeof(std::uint32_t)), block.header->block_version);
+          event_ids.emplace_back( odin.runNumber(), odin.eventNumber() );
+          event_mask[i_event] = 1;
+        }
       }
     }
 
-    auto const allen_type = bank_ids[bank_type];
-    if ( bank_type >= LHCb::RawBank::LastType || allen_type == -1 || !bank_types.count( BankTypes{allen_type} ) ) {
-      prev_type = bank_type;
+    if ( !bank_types.count( allen_type ) ) {
+      prev_type = allen_type;
     } else {
-      if ( bank_type != prev_type ) {
+      if ( allen_type != prev_type ) {
         bank_index = 1;
-        prev_type  = bank_type;
+        prev_type  = allen_type;
       }
 
-      auto        allen_type    = bank_ids[bank_type];
-      auto&       slice         = std::get<0>( slices[allen_type][slice_index] )[0];
-      auto const& event_offsets = std::get<2>( slices[allen_type][slice_index] );
+      auto        sd_index      = to_integral(allen_type);
+      auto&       slice         = std::get<0>( slices[sd_index][slice_index] )[0];
+      auto const& event_offsets = std::get<2>( slices[sd_index][slice_index] );
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
@@ -324,7 +314,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
         // - offsets to individual banks
         // - bank data
 
-        auto preamble_words = 2 + banks_count[subdetector_index];
+        auto preamble_words = 2 + mfp_count[sd_index];
 
         // Initialize point to write from offset of previous set
         // All bank offsets are uit32_t so cast to that type
@@ -335,7 +325,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
         if ( bank_index == 1 ) {
           // Write the number of banks
-          banks_write[0]         = banks_count[subdetector_index];
+          banks_write[0]         = mfp_count[sd_index];
           banks_offsets_write[0] = 0;
         }
 
@@ -349,7 +339,7 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 
         // Write sourceID; offset in 32bit words
         auto word_offset         = offset / sizeof( uint32_t );
-        banks_write[word_offset] = block.header->src_id;
+        banks_write[word_offset] = source_id;
 
         // Write bank data
         std::memcpy( banks_write + word_offset + 1, block.payload + source_offsets[i_event], frag_size );
@@ -362,19 +352,19 @@ bool MEP::transpose_event( Allen::Slices& slices, int const slice_index, std::ve
 }
 
 std::tuple<bool, bool, size_t> MEP::transpose_events(
-    Allen::Slices& slices, int const slice_index, std::vector<int> const& bank_ids,
-    std::unordered_set<BankTypes> const& bank_types, std::array<unsigned int, Allen::NSubDetectors> const& banks_count,
-    EventIDs& event_ids, EB::MEP const* mep, MEP::Blocks const& blocks,
+    Allen::Slices& slices, int const slice_index,
+    std::unordered_set<BankTypes> const& bank_types, std::array<unsigned int, NBankTypes> const& mfp_count,
+    EventIDs& event_ids, std::vector<char>& event_mask, EB::MEP const* mep, MEP::Blocks const& blocks,
     MEP::SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
   auto [event_start, event_end] = interval;
 
   bool success = true;
 
-  auto to_transpose = allen_offsets( slices, slice_index, bank_ids, bank_types, banks_count, blocks, source_offsets,
+  auto to_transpose = allen_offsets( slices, slice_index, bank_types, mfp_count, blocks, source_offsets,
                                      interval, split_by_run );
 
-  transpose_event( slices, slice_index, bank_ids, bank_types, banks_count, event_ids, mep, blocks,
-                   source_offsets, {event_start, event_start + to_transpose} );
+  transpose_mep( slices, slice_index, bank_types, mfp_count, event_ids, event_mask, mep, blocks,
+                 source_offsets, {event_start, event_start + to_transpose} );
 
   return {success, to_transpose != ( event_end - event_start ), to_transpose};
 }
diff --git a/cmake/MooreOnlineDependencies.cmake b/cmake/MooreOnlineDependencies.cmake
index 71c0354a0..2b958d319 100644
--- a/cmake/MooreOnlineDependencies.cmake
+++ b/cmake/MooreOnlineDependencies.cmake
@@ -34,4 +34,6 @@ endif()
 # -- Private dependencies
 if(WITH_MooreOnline_PRIVATE_DEPENDENCIES)
     find_package(Python REQUIRED Interpreter)
+
+    find_package(MPI QUIET COMPONENTS C CXX)
 endif()
-- 
GitLab


From 32b3efab8dfc9c24236f41c797f13d4548b3fec7 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 29 Nov 2021 15:25:55 +0100
Subject: [PATCH 059/120] Fix build

---
 AllenOnline/src/AllenConfiguration.h | 6 ++----
 AllenOnline/src/TransposeMEP.cpp     | 5 ++---
 2 files changed, 4 insertions(+), 7 deletions(-)

diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index 330f77dad..42120c37b 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -59,7 +59,7 @@ public:
         throw GaudiException{"Allen JSON configuration file does not exist or is not a regular file", name(), StatusCode::FAILURE};
       }
       json.set(json_file);
-    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
+    }};
   Gaudi::Property<size_t> nLines{this, "NLines", 0ul};
   Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/parameters",
     [this](auto &) -> void {
@@ -69,13 +69,11 @@ public:
         throw GaudiException{"Allen parameter directory file does not exist or is not a directory", name(), StatusCode::FAILURE};
       }
       paramDir.set(dir);
-    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
-  }
+    }};
   Gaudi::Property<std::string> sequence {this, "Sequence", "hlt1_pp_default"};
 
   Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
   Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
   Gaudi::Property<bool> partitionBuffers {this, "PartitionBuffers", true};
   Gaudi::Property<std::string> partition {this, "Partition", ""};
-  Gaudi::Property<unsigned> verbosity {this, "Verbosity", 3};
 };
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index b5af75cf7..cbd6e7bc9 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -57,10 +57,9 @@ void MEP::find_blocks( EB::MEP const* mep, Blocks& blocks ) {
   auto const*         src_ids = mep->header.src_ids();
   std::vector<size_t> perm( tmp.size() );
   std::iota( perm.begin(), perm.end(), 0U );
-  std::sort( perm.begin(), perm.end(), [&tmp, src_ids]( size_t a, size_t b ) {
+  std::sort( perm.begin(), perm.end(), [src_ids]( size_t a, size_t b ) {
     return src_ids[a] < src_ids[b];
-    }
-  } );
+    });
 
   for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) { blocks[i_block] = tmp[perm[i_block]]; }
 }
-- 
GitLab


From 3f5df0e1f7efc1e32c951667ca43ae22c9821e14 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 29 Nov 2021 15:26:05 +0100
Subject: [PATCH 060/120] Follow changes in Online

---
 AllenOnline/AllenOnline/MBMOutput.h            | 2 --
 AllenOnline/application/test_host_register.cpp | 1 +
 2 files changed, 1 insertion(+), 2 deletions(-)

diff --git a/AllenOnline/AllenOnline/MBMOutput.h b/AllenOnline/AllenOnline/MBMOutput.h
index 3def45847..3e54d53e7 100644
--- a/AllenOnline/AllenOnline/MBMOutput.h
+++ b/AllenOnline/AllenOnline/MBMOutput.h
@@ -15,8 +15,6 @@
 
 #include <Allen/OutputHandler.h>
 
-typedef void* BMID;
-
 class MBMOutput final : public OutputHandler {
 public:
   MBMOutput(
diff --git a/AllenOnline/application/test_host_register.cpp b/AllenOnline/application/test_host_register.cpp
index cd166a592..73be9c8b6 100644
--- a/AllenOnline/application/test_host_register.cpp
+++ b/AllenOnline/application/test_host_register.cpp
@@ -9,6 +9,7 @@
 * or submit itself to any jurisdiction.                                       *
 \*****************************************************************************/
 #include <Backend/BackendCommon.h>
+#include <RTL/rtl.h>
 #include <MBM/bmdef.h>
 
 #include <iostream>
-- 
GitLab


From ae8dfa25b03dd7b8139d7c8673e98a7f9fd9f905 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 30 Nov 2021 20:23:16 +0100
Subject: [PATCH 061/120] Make hwloc optional

---
 AllenOnline/CMakeLists.txt          | 8 +++-----
 cmake/MooreOnlineDependencies.cmake | 3 +++
 2 files changed, 6 insertions(+), 5 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 7b27143b3..46aaa8dd1 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -52,11 +52,8 @@ gaudi_add_module(AllenOnlineComp
         rt)
 
 # MPI
-if (MPI_FOUND)
+if (MPI_FOUND AND hwloc_FOUND)
   message(STATUS "Found MPI ${MPI_CXX_VERSION}: ${MPI_CXX_COMPILER}")
-  #hwloc is also needed
-  find_package(PkgConfig REQUIRED)
-  pkg_check_modules(hwloc REQUIRED IMPORTED_TARGET hwloc)
 
   # open-pal library is also needed...
   set(MPI_LIBDIRS)
@@ -76,7 +73,8 @@ if (MPI_FOUND)
       LINK
         AllenOnline LHCb::MDFLib
         Boost::program_options
-        MPI::MPI_CXX PkgConfig::hwloc
+        MPI::MPI_CXX
+        PkgConfig::hwloc
         ${open_pal_lib})
    target_include_directories(allen_mpi_send PRIVATE ${MPI_CXX_INCLUDE_DIRS})
    target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
diff --git a/cmake/MooreOnlineDependencies.cmake b/cmake/MooreOnlineDependencies.cmake
index 2b958d319..4d263fdb1 100644
--- a/cmake/MooreOnlineDependencies.cmake
+++ b/cmake/MooreOnlineDependencies.cmake
@@ -36,4 +36,7 @@ if(WITH_MooreOnline_PRIVATE_DEPENDENCIES)
     find_package(Python REQUIRED Interpreter)
 
     find_package(MPI QUIET COMPONENTS C CXX)
+
+    find_package(PkgConfig REQUIRED)
+    pkg_check_modules(hwloc QUIET IMPORTED_TARGET hwloc)
 endif()
-- 
GitLab


From 8b1f9035abe3ab5b0e610e0c2794641ed2cc52a0 Mon Sep 17 00:00:00 2001
From: rta <rta@tdeb02.lbdaq.cern.ch>
Date: Wed, 1 Dec 2021 18:51:15 +0100
Subject: [PATCH 062/120] Pickup the correct sequence from JSON

---
 AllenOnline/options/Allen.py         | 5 ++---
 AllenOnline/src/AllenApplication.cpp | 4 ++--
 AllenOnline/src/AllenConfiguration.h | 4 +---
 AllenOnline/src/MBMOutput.cpp        | 3 ++-
 4 files changed, 7 insertions(+), 9 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 1224d8de6..f963e0b34 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -48,7 +48,7 @@ CondDB().Upgrade = True
 # be done internally to Allen and retrieved somehow
 n_lines = 0
 # json_file = "/daqarea1/fest/allen/configuration/passthrough.json"
-json_file = "/group/hlt/fest_202106/dev-dir/MooreOnline/AllenOnline/options/passthrough.json"
+json_file = os.path.expandvars("${ALLEN_INSTALL_DIR}/constants/passthrough.json")
 with open(json_file) as jf:
     json_config = json.load(jf)
     gs = json_config.get('gather_selections', {})
@@ -79,7 +79,6 @@ allen_conf.NThreads = 8
 # allen_conf.Device = "01:00.0"
 allen_conf.Device = "0"
 allen_conf.JSON = json_file
-allen_conf.Sequence = "passthrough"
 allen_conf.NLines = n_lines
 allen_conf.OutputChecksum = False
 allen_conf.ParamDir = "/group/hlt/fest_202106/dev-dir/Allen/input/parameters"
@@ -87,7 +86,7 @@ allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
 allen_conf.PartitionID = partition_id
 allen_conf.EnableRunChanges = False
-allen_conf.Verbosity = 5
+#allen_conf.Verbosity = 5
 
 if run_online:
     allen_conf.Output = "mbm://Output"
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 4ac798145..47c19af1d 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -307,11 +307,11 @@ void AllenApplication::allenLoop()
   std::map<std::string, std::string> allen_options = {{"v", std::to_string(6 - m_config->outputLevel())},
                                                       {"t", m_allenConfig->nThreads.toString()},
                                                       {"params", paramDir},
-                                                      {"configuration", m_json},
                                                       {"device", m_allenConfig->device.value()},
                                                       {"s", std::to_string(m_nSlices)},
                                                       {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())},
-                                                      {"sequence", m_allenConfig->sequence.value()}};
+                                                      {"sequence", m_allenConfig->json.value()},
+                                                      {"run-from-json", "1"}};
   allen(allen_options, m_updater, m_provider, m_output.get(), m_zmqSvc.get(), m_controlConnection);
 }
 
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index 42120c37b..9f690c773 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -51,7 +51,7 @@ public:
   Gaudi::Property<std::string> output {this, "Output", ""};
   Gaudi::Property<bool> runChanges {this, "EnableRunChanges", true};
   Gaudi::Property<std::string> device {this, "Device", "0"};
-  Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_PROJECT_ROOT}/configuration/constants/default.json",
+  Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_INSTALL_DIR}/constants/hlt1_pp_default.json",
     [this](auto &) -> void {
       auto const json_file = resolveEnvVars(json.value());
       fs::path j{json_file};
@@ -70,8 +70,6 @@ public:
       }
       paramDir.set(dir);
     }};
-  Gaudi::Property<std::string> sequence {this, "Sequence", "hlt1_pp_default"};
-
   Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
   Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
   Gaudi::Property<bool> partitionBuffers {this, "PartitionBuffers", true};
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 3fef3c39d..7f2feb196 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -1,4 +1,3 @@
-
 /*****************************************************************************\
 * (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
 *                                                                             *
@@ -48,6 +47,8 @@ std::tuple<size_t, gsl::span<char>> MBMOutput::buffer(size_t buffer_size)
       m_buffer = {reinterpret_cast<char*>(buf), static_cast<events_size>(buffer_size)};
       return {0, m_buffer};
     }
+  } else {
+    m_logger->error("%s failed to get space of size %d in buffer %s", m_processName.c_str(), buffer_size, m_bufferName.c_str());
   }
   return {0, {}};
 }
-- 
GitLab


From 86a7b5a35f0ab744bede2670b799042ead1ec631 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 2 Dec 2021 14:57:38 +0100
Subject: [PATCH 063/120] Fix optional MPI/hwloc build

---
 AllenOnline/src/MEPProvider.cpp | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 3fd2b95a9..6bd924fbe 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -1550,6 +1550,7 @@ void MEPProvider::transpose(int thread_id)
 }
 
 StatusCode MEPProvider::numa_membind(char const* mem, size_t size, int const numa_node) const {
+#ifdef HAVE_MPI
   auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
   auto s = hwloc_set_area_membind(
     m_topology, mem, size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET | HWLOC_MEMBIND_STRICT | HWLOC_MEMBIND_MIGRATE);
@@ -1561,4 +1562,7 @@ StatusCode MEPProvider::numa_membind(char const* mem, size_t size, int const num
   else {
     return StatusCode::SUCCESS;
   }
+#else
+  return StatusCode::FAILURE;
+#endif
 }
-- 
GitLab


From b2a7ffd1b870fefcaa6589804128b2aa6469ab82 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 6 Dec 2021 14:43:30 +0100
Subject: [PATCH 064/120] Adapt to small changes for monitoring in Allen

---
 AllenOnline/AllenOnline/MBMOutput.h  | 8 ++++----
 AllenOnline/src/AllenApplication.cpp | 3 +++
 AllenOnline/src/MBMOutput.cpp        | 5 ++---
 3 files changed, 9 insertions(+), 7 deletions(-)

diff --git a/AllenOnline/AllenOnline/MBMOutput.h b/AllenOnline/AllenOnline/MBMOutput.h
index 3e54d53e7..d666d35a5 100644
--- a/AllenOnline/AllenOnline/MBMOutput.h
+++ b/AllenOnline/AllenOnline/MBMOutput.h
@@ -29,15 +29,15 @@ public:
   /// Callback when space is availible (not really used)
   static int spaceCallback(void* /* param */);
 
+  bool start();
+
 protected:
 
-  std::tuple<size_t, gsl::span<char>> buffer(size_t buffer_size) override;
+  std::tuple<size_t, gsl::span<char>> buffer(size_t buffer_size, size_t) override;
 
   bool write_buffer(size_t) override;
 
-  bool start() override;
-
-  bool stop() override;
+  void output_done() override;
 
   void cancel() override;
 
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 47c19af1d..f728ebea7 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -279,6 +279,9 @@ int AllenApplication::startApplication()
     }
   }
 
+  auto* mbm_output = dynamic_cast<MBMOutput*>(m_output.get());
+  if (mbm_output != nullptr) mbm_output->start();
+
   fireIncident("DAQ_RUNNING");
   fireIncident("APP_RUNNING");
   return Online::ONLINE_OK;
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 7f2feb196..7aa4a45ca 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -33,7 +33,7 @@ int MBMOutput::spaceCallback(void* /* param */)
   return MBM_NORMAL;
 }
 
-std::tuple<size_t, gsl::span<char>> MBMOutput::buffer(size_t buffer_size)
+std::tuple<size_t, gsl::span<char>> MBMOutput::buffer(size_t buffer_size, size_t)
 {
   if (m_cancelled) {
     return {0, {}};
@@ -103,11 +103,10 @@ void MBMOutput::cancel() {
   m_cancelled = true;
 }
 
-bool MBMOutput::stop()
+void MBMOutput::output_done()
 {
   m_cancelled = false;
   if (m_bmID != MBM_INV_DESC) {
     ::mbm_exclude(m_bmID);
   }
-  return true;
 }
-- 
GitLab


From e92aa20109763ae59d4889acd739dec1231e7d45 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 6 Dec 2021 14:43:53 +0100
Subject: [PATCH 065/120] Fix filling of event masks; use proper interval

---
 AllenOnline/src/MEPProvider.cpp  | 5 +----
 AllenOnline/src/TransposeMEP.cpp | 4 ++--
 2 files changed, 3 insertions(+), 6 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 6bd924fbe..4b400c5d3 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -334,6 +334,7 @@ StatusCode MEPProvider::initialize()
   // Allocate space to store event ids
   for (size_t n = 0; n < m_nslices.value(); ++n) {
     m_event_ids[n].reserve(m_events_per_slice);
+    m_event_masks[n].resize(m_events_per_slice, 0);
   }
 
   return sc;
@@ -681,10 +682,6 @@ bool MEPProvider::allocate_storage(size_t i_read)
     assert(slice.packing_factor == m_packing_factor);
   }
 
-  for (auto& mask : m_event_masks) {
-    mask.resize(m_packing_factor, 0);
-  }
-
   size_t const eps = m_events_per_slice.value();
   auto n_interval = m_packing_factor / eps;
   auto rest = m_packing_factor % eps;
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index cbd6e7bc9..d30cd8d30 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -284,13 +284,13 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
       for ( uint16_t i_event = start_event; i_event < end_event; ++i_event ) {
         if (block.bank_types[i_event] != LHCb::RawBank::ODIN) {
           event_ids.emplace_back( 0, 0 );
-          event_mask[i_event] = 0;
+          event_mask[i_event - start_event] = 0;
         }
         else {
           auto const odin = decode_odin(block.payload, source_offsets[i_event],
                                         (bank_sizes[i_event] / sizeof(std::uint32_t)), block.header->block_version);
           event_ids.emplace_back( odin.runNumber(), odin.eventNumber() );
-          event_mask[i_event] = 1;
+          event_mask[i_event - start_event] = 1;
         }
       }
     }
-- 
GitLab


From ad682f777cf8509eacf43411b81b483c50d5504b Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 6 Dec 2021 16:35:44 +0100
Subject: [PATCH 066/120] Fix filling of EventIDs and event mask

---
 AllenOnline/src/MEPProvider.cpp  |  4 ++--
 AllenOnline/src/TransposeMEP.cpp | 38 +++++++++++++++++---------------
 2 files changed, 22 insertions(+), 20 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 4b400c5d3..943ebc59b 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -333,8 +333,8 @@ StatusCode MEPProvider::initialize()
 
   // Allocate space to store event ids
   for (size_t n = 0; n < m_nslices.value(); ++n) {
-    m_event_ids[n].reserve(m_events_per_slice);
-    m_event_masks[n].resize(m_events_per_slice, 0);
+    m_event_ids[n].reserve(m_events_per_slice.value());
+    m_event_masks[n].resize(m_events_per_slice.value(), 0);
   }
 
   return sc;
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index d30cd8d30..a4e1e672a 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -190,25 +190,27 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       // decode ODIN bank to obtain run and event numbers
       unsigned fragment_offset = 0;
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
-        if ( i_event >= event_start && block.bank_types[i_event] == LHCb::RawBank::ODIN) {
-          auto const odin = decode_odin(block.payload, fragment_offset,
-                                        bank_sizes[i_event], block.header->block_version);
-          unsigned const odin_run_number = odin.runNumber();
-          auto const event_number = odin.eventNumber();
-
-          // if splitting by run, check all events have same run number
-          if ( !run_number ) {
-            run_number = odin_run_number;
-          } else if ( split_by_run && run_number && odin_run_number != *run_number ) {
-            event_end = i_event;
-            break;
+        if ( i_event >= event_start) {
+          if (block.bank_types[i_event] == LHCb::RawBank::ODIN) {
+            auto const odin = decode_odin(block.payload, fragment_offset,
+                                          bank_sizes[i_event], block.header->block_version);
+            unsigned const odin_run_number = odin.runNumber();
+            auto const event_number = odin.eventNumber();
+
+            // if splitting by run, check all events have same run number
+            if ( !run_number ) {
+              run_number = odin_run_number;
+            } else if ( split_by_run && run_number && odin_run_number != *run_number ) {
+              event_end = i_event;
+              break;
+            }
+            event_ids.emplace_back( odin_run_number, event_number );
+            event_mask[i_event - event_start] = 1;
+          }
+          else {
+            event_ids.emplace_back( 0, 0 );
+            event_mask[i_event - event_start] = 0;
           }
-          event_ids.emplace_back( odin_run_number, event_number );
-          event_mask[i_event - event_start] = 1;
-        }
-        else {
-          event_ids.emplace_back( 0, 0 );
-          event_mask[i_event - event_start] = 0;
         }
         fragment_offset += bank_sizes[i_event] + EB::get_padding(bank_sizes[i_event], 1 << align);
       }
-- 
GitLab


From 5d6fdbae8c05198e6e8d5d32e23c97eb24692ac8 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Sat, 30 Oct 2021 14:59:07 +0200
Subject: [PATCH 067/120] Make MBMOutput a service and add monitoring counters

---
 AllenOnline/AllenOnline/MEPProvider.h        |   5 +
 AllenOnline/options/Allen.py                 |  23 ++--
 AllenOnline/src/AllenApplication.cpp         | 127 ++++++++++-------
 AllenOnline/src/AllenApplication.h           |  36 ++---
 AllenOnline/src/MBMOutput.cpp                | 135 +++++++++++++++----
 AllenOnline/{AllenOnline => src}/MBMOutput.h |  48 ++++---
 AllenOnline/src/MEPProvider.cpp              |  35 ++++-
 7 files changed, 288 insertions(+), 121 deletions(-)
 rename AllenOnline/{AllenOnline => src}/MBMOutput.h (61%)

diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/AllenOnline/MEPProvider.h
index 35489ddd3..f32b6b497 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/AllenOnline/MEPProvider.h
@@ -21,6 +21,7 @@
 #include <fcntl.h>
 
 #include <Gaudi/Property.h>
+#include <Gaudi/Accumulators.h>
 #include <Kernel/meta_enum.h>
 
 #include <GaudiKernel/Service.h>
@@ -319,4 +320,8 @@ private:
   Gaudi::Property<std::vector<std::string>> m_requests {this, "Requests", {}};
   Gaudi::Property<bool> m_thread_per_buffer {this, "ThreadPerBuffer", true};
 
+  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mepsInput;
+  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsInput;
+  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbInput;
+
 };
diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index f963e0b34..4dd3705eb 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -85,11 +85,15 @@ allen_conf.ParamDir = "/group/hlt/fest_202106/dev-dir/Allen/input/parameters"
 allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
 allen_conf.PartitionID = partition_id
-allen_conf.EnableRunChanges = False
+allen_conf.EnableRunChanges = True
 #allen_conf.Verbosity = 5
 
 if run_online:
-    allen_conf.Output = "mbm://Output"
+    from Configurables import Allen__MBMOutput as MBMOutput
+    output_svc = MBMOutput()
+    output_svc.Checksum = True
+    output_svc.BufferName = "Output"
+    app.ExtSvc += [output_svc]
 elif integration_test:
     allen_conf.Output = "tcp://192.168.1.101:35000"
 else:
@@ -122,14 +126,16 @@ else:
     mep_provider.Connections = sorted([os.path.join(mep_dir, mep_file)
                                        for mep_file in os.listdir(mep_dir)
                                        if mep_file.endswith('.mep')])
-    mep_provider.LoopOnMEPs = False
+    mep_provider.LoopOnMEPs = True
 
 monSvc = MonitorSvc('MonitorSvc')
-monSvc.ExpandNameInfix = '<proc>'
+monSvc.ExpandNameInfix = '<proc>/'
 monSvc.ExpandCounterServices = True
 monSvc.UniqueServiceNames = True
+monSvc.UseDStoreNames = True
 monSvc.PartitionName = partition
-
+monSvc.DimUpdateInterval = 5
+monSvc.OutputLevel = 2
 
 setup_allen_non_event_data_service()
 
@@ -143,9 +149,10 @@ messageSvc.OutputLevel = output_level
 
 
 # Add the services that will produce the non-event-data
-app.ExtSvc += [
-    monSvc
-]
+app.ExtSvc = [
+    monSvc,
+    "OnlMonitorSink"
+] + app.ExtSvc
 
 # Some extra stuff for timing table
 ApplicationMgr().EvtSel = "NONE"
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index f728ebea7..d4c10513f 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -41,7 +41,6 @@
 #include <Allen/Provider.h>
 
 #include <AllenOnline/MEPProvider.h>
-#include <AllenOnline/MBMOutput.h>
 
 #ifdef HAVE_MPI
 #include <MPIConfig.h>
@@ -49,6 +48,7 @@
 
 #include "AllenConfiguration.h"
 #include "AllenApplication.h"
+#include "MBMOutput.h"
 
 // #include "EBProvider.h"
 
@@ -60,13 +60,6 @@ namespace {
 /// Factory instantiation
 DECLARE_COMPONENT(AllenApplication)
 
-/// Reset counters at start
-void AllenApplication::monitor_t::reset()
-{
-  mepsIn = 0;
-  eventsOut = 0;
-}
-
 /// Specialized constructor
 AllenApplication::AllenApplication(Options opts) : OnlineApplication(opts) {}
 
@@ -85,8 +78,17 @@ int AllenApplication::stop()
   m_zmqSvc->send(*m_allenControl, m_allenConfig->stopTimeout.value());
 
   // This will stop input
-  auto sc = OnlineApplication::stop();
-  if (!sc) return sc;
+  fireIncident("DAQ_STOP");
+
+
+  StatusCode sc = app->stop();
+  if ( !sc.isSuccess() )   {
+    m_logger->error("%s Class %d application: Failed to execute the "
+                    "transition to state %s",
+                    "app->stop()", 1 , stateName(ST_READY));
+    fireIncident("DAQ_ERROR");
+    return declareState(ST_ERROR);
+  }
 
   zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
   m_zmqSvc->poll(&items[0], 1, -1);
@@ -102,7 +104,10 @@ int AllenApplication::stop()
   }
 
   m_provider->release_buffers();
-  return sc;
+
+  fireIncident("DAQ_STOPPED");
+  fireIncident("APP_STOPPED");
+  return declareState(ST_READY);
 }
 
 /// Cancel the application: Cancel IO request/Event loop
@@ -134,21 +139,6 @@ int AllenApplication::configureApplication()
 
   SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
 
-  if (!m_config->monitorType.empty()) {
-
-    m_monMEPs.reset(new Service("MEPs", sloc));
-    m_monEvents.reset(new Service("Events", sloc));
-
-    m_monSvc = sloc->service<IMonitorSvc>(m_config->monitorType);
-    if (!m_monSvc.get()) {
-      m_logger->error("Cannot access monitoring service of type %s.", m_config->monitorType.c_str());
-      return Online::ONLINE_ERROR;
-    }
-    m_monSvc->declareInfo("IN", m_monitor.mepsIn, "Number of MEPs received for processing", m_monMEPs);
-    m_monSvc->declareInfo("OUT", m_monitor.mepsDone, "Number of MEPs fully processed", m_monMEPs);
-    m_monSvc->declareInfo("OUT", m_monitor.eventsOut, "Number of events fully output", m_monEvents);
-  }
-
   m_allenConfig = sloc->service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
   if (!m_allenConfig) {
     m_logger->throwError("Failed to retrieve AllenConfiguration.");
@@ -193,6 +183,14 @@ int AllenApplication::configureApplication()
     }
   }
 
+  if (!m_config->monitorType.empty()) {
+    m_monSvc = sloc->service<IGauchoMonitorSvc>(m_config->monitorType);
+    if (!m_monSvc) {
+      m_logger->error("Cannot access monitoring service of type %s.", m_config->monitorType.c_str());
+      return Online::ONLINE_ERROR;
+    }
+  }
+
   auto* provider = dynamic_cast<IInputProvider*>(mepProvider.get());
   if (provider == nullptr) {
     m_logger->error("Failed to cast MEPProvider");
@@ -207,6 +205,7 @@ int AllenApplication::configureApplication()
   m_output = makeOutput();
   if (!m_allenConfig->output.value().empty() && !m_output) {
     m_logger->error("Failed to create output for "s + m_allenConfig->output.value());
+    return Online::ONLINE_ERROR;
   }
 
   m_allenControl = m_zmqSvc->socket(zmq::PAIR);
@@ -246,13 +245,6 @@ int AllenApplication::finalizeApplication()
     }
   }
 
-  if (m_monSvc.get()) {
-    m_monSvc->undeclareAll(m_monMEPs);
-    m_monSvc->undeclareAll(m_monEvents);
-    m_monSvc.reset();
-  }
-  m_monMEPs.reset();
-  m_monEvents.reset();
   return OnlineApplication::finalizeApplication();
 }
 
@@ -279,9 +271,6 @@ int AllenApplication::startApplication()
     }
   }
 
-  auto* mbm_output = dynamic_cast<MBMOutput*>(m_output.get());
-  if (mbm_output != nullptr) mbm_output->start();
-
   fireIncident("DAQ_RUNNING");
   fireIncident("APP_RUNNING");
   return Online::ONLINE_OK;
@@ -315,18 +304,50 @@ void AllenApplication::allenLoop()
                                                       {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())},
                                                       {"sequence", m_allenConfig->json.value()},
                                                       {"run-from-json", "1"}};
-  allen(allen_options, m_updater, m_provider, m_output.get(), m_zmqSvc.get(), m_controlConnection);
+  allen(allen_options, this, m_provider, m_output, m_zmqSvc.get(), m_controlConnection);
+}
+
+void AllenApplication::update(unsigned long run)
+{
+  // Monitoring run change
+  if (m_runNumber == 0) {
+    m_runNumber = run;
+    m_monSvc->setRunNo(run);
+  }
+  else if (run > m_runNumber) {
+    m_monSvc->update(m_runNumber).ignore();
+    m_monSvc->resetHistos(nullptr);
+    m_monSvc->setRunNo(run);
+    m_runNumber = run;
+  }
+
+  // Update conditions by forwarding to the real updater
+  m_updater->update(run);
+}
+
+void AllenApplication::registerConsumer(std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c)
+{
+  m_updater->registerConsumer(id, std::move(c));
 }
 
-std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
+void AllenApplication::registerProducer(std::string const& id, Allen::NonEventData::Producer p)
+{
+  m_updater->registerProducer(id, std::move(p));
+}
+
+
+OutputHandler* AllenApplication::makeOutput()
 {
   size_t n_lines = m_allenConfig->nLines.value();
   if (n_lines == 0) {
     m_logger->warning("No lines configured in Allen sequence");
   }
 
+  SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
+  auto outputSvc = sloc->service("Allen::MBMOutput/MBMOutput");
+
   auto output = m_allenConfig->output.value();
-  if (output.empty()) {
+  if (!outputSvc && output.empty()) {
     m_logger->warning("No output is configured, selected events will be lost");
     return {};
   }
@@ -335,11 +356,11 @@ std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
   auto p = output.find("://");
   std::string output_type;
 
-  if (p == std::string::npos) {
+  if (!output.empty() && p == std::string::npos) {
     output_type = "file";
     connection = output;
   }
-  else {
+  else if (!output.empty()) {
     output_type = output.substr(0, p);
     connection = output.substr(p + 3, std::string::npos);
   }
@@ -347,9 +368,10 @@ std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
   if (output_type == "file" || output_type == "tcp") {
     std::map<std::string, std::string> options = {{"configuration", m_json},
                                                   {"output-file", output}};
-    return Allen::output_handler(m_provider.get(), m_zmqSvc.get(), std::move(options));
+    m_outputHolder = Allen::output_handler(m_provider.get(), m_zmqSvc.get(), std::move(options));
+    return m_outputHolder.get();
   }
-  else if (output_type == "mbm") {
+  else if (outputSvc) {
     if (m_allenConfig->partitionBuffers.value()) {
       connection += "_";
       auto const partition = m_allenConfig->partition.value();
@@ -358,19 +380,20 @@ std::unique_ptr<OutputHandler> AllenApplication::makeOutput()
       connection += partition.empty() ? hexID.str() : m_allenConfig->partition.value();
     }
 
-    auto output_logger = m_logger->clone("MBMOutput", m_config->outputLevel());
+    if (!outputSvc) {
+      m_logger->error("Failed to retrieve MBMOutput.");
+      return {};
+    }
 
-    return std::make_unique<MBMOutput>(std::move(output_logger),
-                                       m_provider.get(),
-                                       connection,
-                                       RTL::processName(),
-                                       m_allenConfig->partitionID.value(),
-                                       n_lines,
-                                       m_allenConfig->checksum.value());
+    auto* mbmOutput = dynamic_cast<Allen::MBMOutput*>(outputSvc.get());
+    if (mbmOutput == nullptr) {
+      m_logger->error("Failed to cast MBMOutput");
+    }
+    return mbmOutput;
   }
   else {
     m_logger->error("Unknown output type: "s + output_type);
-    return {};
+    return nullptr;
   }
 
 }
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 207f3e583..2eeac8c02 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -28,23 +28,16 @@
 #include <dim/dis.hxx>
 
 #include <GaudiOnline/OnlineApplication.h>
+#include <Gaucho/IGauchoMonitorSvc.h>
 #include <ZeroMQ/IZeroMQSvc.h>
 
+#include <Dumpers/IUpdater.h>
 #include <Allen/OutputHandler.h>
 
 #include "AllenConfiguration.h"
 
-class AllenApplication : public Online::OnlineApplication {
+class AllenApplication : public Online::OnlineApplication, public Allen::NonEventData::IUpdater {
 public:
-  /// Structurte containing all monitoring items
-  struct monitor_t {
-    long mepsIn = 0;
-    long mepsDone = 0;
-    long eventsOut = 0;
-    monitor_t() = default;
-    virtual ~monitor_t() = default;
-    void reset();
-  } m_monitor;
 
   // Specialized constructor
   AllenApplication(Options opts);
@@ -73,12 +66,21 @@ public:
 
   bool initMPI();
 
+  // Updater
+  void update(unsigned long run) override;
+
+  void registerConsumer(std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c) override;
+
+  void registerProducer(std::string const& id, Allen::NonEventData::Producer p) override;
+
 private:
 
-  std::unique_ptr<OutputHandler> makeOutput();
+  OutputHandler* makeOutput();
+
+  void runChange();
 
   /// Reference to the monitoring service
-  SmartIF<IMonitorSvc> m_monSvc;
+  SmartIF<IGauchoMonitorSvc> m_monSvc;
 
   /// Handles to helper service to properly name burst counters
   SmartIF<IService> m_monMEPs;
@@ -88,13 +90,17 @@ private:
   // ZeroMQSvc
   SmartIF<IZeroMQSvc> m_zmqSvc;
 
+  unsigned long m_runNumber = 0;
+
   Allen::NonEventData::IUpdater* m_updater = nullptr;
   SmartIF<AllenConfiguration> m_allenConfig;
 
-  std::string m_controlConnection = "inproc://AllenApplicationControl";
+  std::string const m_controlConnection = "inproc://AllenApplicationControl";
+  std::string const m_runChangeConnection = "inproc://AllenApplicationRunChange";
 
   std::shared_ptr<IInputProvider> m_provider;
-  std::unique_ptr<OutputHandler> m_output;
+  std::unique_ptr<OutputHandler> m_outputHolder;
+  OutputHandler* m_output;
 
   size_t m_nSlices = 0;
 
@@ -105,7 +111,7 @@ private:
   // allen_t m_allenFun = nullptr;
 
   std::thread m_allenThread;
-  std::optional<zmq::socket_t> m_allenControl;
+  std::optional<zmq::socket_t> m_allenControl = std::nullopt;
 
   std::string m_json;
 };
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 7aa4a45ca..b8edef2c0 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -8,37 +8,41 @@
 * granted to it by virtue of its status as an Intergovernmental Organization  *
 * or submit itself to any jurisdiction.                                       *
 \*****************************************************************************/
-#include <AllenOnline/MBMOutput.h>
+#include <Gaudi/Accumulators.h>
+
 #include <MDF/MDFHeader.h>
 #include <MDF/RawEventHelpers.h>
 
-MBMOutput::MBMOutput(
-    std::unique_ptr<RTL::Logger>&& logger,
-    IInputProvider const* inputProvider,
-    std::string bufferName,
-    std::string processName,
-    unsigned partitionID,
-    size_t const nLines,
-    bool const checksum) :
-    OutputHandler {inputProvider, bufferName, nLines, checksum},
-    m_logger{std::move(logger)},
-    m_bufferName {std::move(bufferName)},
-    m_processName{std::move(processName)},
-    m_partitionID{partitionID}
+#include "MBMOutput.h"
+
+Allen::MBMOutput::MBMOutput(std::string name, ISvcLocator* loc)
+  : Service {name, loc}
 {
 }
 
-int MBMOutput::spaceCallback(void* /* param */)
+int Allen::MBMOutput::spaceCallback(void* /* param */)
 {
   return MBM_NORMAL;
 }
 
-std::tuple<size_t, gsl::span<char>> MBMOutput::buffer(size_t buffer_size, size_t)
+std::tuple<size_t, gsl::span<char>> Allen::MBMOutput::buffer(size_t buffer_size, size_t n_events)
 {
   if (m_cancelled) {
     return {0, {}};
   }
 
+  {
+    std::lock_guard<std::mutex> lock{m_mutex};
+    size_t id = 0;
+    if (m_freeIDs.empty()) {
+      id = m_id++;
+    } else {
+      id = m_freeIDs.front();
+      m_freeIDs.pop_front();
+    }
+    m_eventCounts[id] = n_events;
+  }
+
   int* buf = nullptr;
   auto sc = ::mbm_get_space_a(m_bmID, buffer_size, &buf, spaceCallback, this);
   if ( sc == MBM_NORMAL )  {
@@ -48,31 +52,103 @@ std::tuple<size_t, gsl::span<char>> MBMOutput::buffer(size_t buffer_size, size_t
       return {0, m_buffer};
     }
   } else {
-    m_logger->error("%s failed to get space of size %d in buffer %s", m_processName.c_str(), buffer_size, m_bufferName.c_str());
+    error() << m_processName << " failed to get space of size " << buffer_size << " in buffer " << m_bufferName.value() << endmsg;
   }
   return {0, {}};
 }
 
-bool MBMOutput::start()
+
+StatusCode Allen::MBMOutput::initialize()
+{
+  auto sc = Service::initialize();
+  if (!sc) return sc;
+
+  m_allenConfig = serviceLocator()->service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
+  if (!m_allenConfig) {
+    error() << "Failed to retrieve AllenConfiguration." << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  SmartIF<IService> mepProvider = serviceLocator()->service<IService>("MEPProvider");
+  if (!mepProvider.get()) {
+    error() << "Failed to retrieve MEPProvider." << endmsg;
+    return StatusCode::FAILURE;
+  }
+  auto* inputProvider = dynamic_cast<IInputProvider*>(mepProvider.get());
+  if (inputProvider == nullptr) {
+    error() << "Failed to cast MEPProvider" << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  m_processName = RTL::processName();
+
+  size_t nLines = m_allenConfig->nLines.value();
+
+  m_partitionID = m_allenConfig->partitionID.value();
+  std::string connection = m_bufferName.value();
+  if (m_allenConfig->partitionBuffers.value()) {
+    connection += "_";
+    auto const partition = m_allenConfig->partition.value();
+    std::stringstream hexID;
+    hexID << std::hex << m_partitionID;
+    connection += partition.empty() ? hexID.str() : partition;
+  }
+
+  auto eventsSvc = dynamic_cast<Service*>(service<IService>("Service/Events", true).get());
+  if (!eventsSvc) {
+    error() << "Failed to obtain Events service for monitoring" << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  auto burstsSvc = dynamic_cast<Service*>(service<IService>("Service/Bursts", true).get());
+  if (!eventsSvc) {
+    error() << "Failed to obtain Bursts service for monitoring" << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  m_burstsOutput = std::make_unique<Gaudi::Accumulators::Counter<>>(burstsSvc, "OUT");
+  m_eventsOutput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "OUT");
+  m_mbOutput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "MB_OUT");
+
+  init(inputProvider, connection, nLines, m_checksum.value());
+  return StatusCode::SUCCESS;
+}
+
+StatusCode Allen::MBMOutput::start()
 {
+  auto sc = Service::start();
+  if (!sc) return sc;
+
   m_cancelled = false;
-  m_bmID = ::mbm_include_write(m_bufferName.c_str(), m_processName.c_str(), m_partitionID, BM_COM_FIFO);
+  m_bmID = ::mbm_include_write(connection().c_str(), m_processName.c_str(), m_partitionID, BM_COM_FIFO);
   if (m_bmID == MBM_INV_DESC)   {
-    return m_logger->error("MBMOutput: failed to connect to MBM buffer %s!",
-                           m_bufferName.c_str());
-  } else {
-    m_logger->info("MBMOutput: connected to MBM buffer %s",
-                   m_bufferName.c_str());
-    return true;
+    error() << "MBMOutput: failed to connect to MBM buffer " << connection() << endmsg;
+    return StatusCode::FAILURE;
+    } else {
+    info() << "MBMOutput: connected to MBM buffer " << connection() << endmsg;
+    return StatusCode::SUCCESS;
   }
 }
 
-bool MBMOutput::write_buffer(size_t)
+bool Allen::MBMOutput::write_buffer(size_t id)
 {
   if (m_cancelled) {
     return false;
   }
 
+  size_t n_events = 0;
+  {
+    std::lock_guard<std::mutex> lock{m_mutex};
+    auto it = m_eventCounts.find(id);
+    n_events = it->second;
+    m_eventCounts.erase(it);
+    m_freeIDs.push_back(id);
+  }
+
+  (*m_burstsOutput) += 1;
+  (*m_eventsOutput) += n_events;
+  (*m_mbOutput) += (2 * m_buffer.size_bytes() + 1) / (2 * 1024 * 1024);
+
   unsigned int mask[] = { m_partitionID, ~0x0U, ~0x0U, ~0x0U };
   void* free_address = nullptr;
   long free_len = 0;
@@ -92,18 +168,19 @@ bool MBMOutput::write_buffer(size_t)
       return true;
     }
   } else {
-    return m_logger->error("MBMOutput: failed to write buffer %d!", sc);
+    error() << "MBMOutput: failed to write buffer " << sc << endmsg;
+    return false;
   }
 }
 
-void MBMOutput::cancel() {
+void Allen::MBMOutput::cancel() {
   if (!m_cancelled && m_bmID != MBM_INV_DESC) {
     ::mbm_cancel_request(m_bmID);
   }
   m_cancelled = true;
 }
 
-void MBMOutput::output_done()
+void Allen::MBMOutput::output_done()
 {
   m_cancelled = false;
   if (m_bmID != MBM_INV_DESC) {
diff --git a/AllenOnline/AllenOnline/MBMOutput.h b/AllenOnline/src/MBMOutput.h
similarity index 61%
rename from AllenOnline/AllenOnline/MBMOutput.h
rename to AllenOnline/src/MBMOutput.h
index d666d35a5..454d9a44a 100644
--- a/AllenOnline/AllenOnline/MBMOutput.h
+++ b/AllenOnline/src/MBMOutput.h
@@ -15,25 +15,28 @@
 
 #include <Allen/OutputHandler.h>
 
-class MBMOutput final : public OutputHandler {
+#include <GaudiKernel/Service.h>
+
+#include "AllenConfiguration.h"
+
+namespace Allen {
+
+class MBMOutput final : public Service, public OutputHandler {
 public:
-  MBMOutput(
-    std::unique_ptr<RTL::Logger>&& logger,
-    IInputProvider const* inputProvider,
-    std::string bufferName,
-    std::string processName,
-    unsigned partitionID,
-    size_t const nLines,
-    bool const checksum = true);
+  MBMOutput(std::string name, ISvcLocator* loc);
+
+  // std::unique_ptr<RTL::Logger>&& logger);
 
   /// Callback when space is availible (not really used)
   static int spaceCallback(void* /* param */);
 
-  bool start();
+  StatusCode initialize() override;
+
+  StatusCode start() override;
 
 protected:
 
-  std::tuple<size_t, gsl::span<char>> buffer(size_t buffer_size, size_t) override;
+  std::tuple<size_t, gsl::span<char>> buffer(size_t buffer_size, size_t n_events) override;
 
   bool write_buffer(size_t) override;
 
@@ -42,14 +45,17 @@ protected:
   void cancel() override;
 
 private:
-  // Logger
-  std::unique_ptr<RTL::Logger> m_logger;
+
+  SmartIF<AllenConfiguration> m_allenConfig;
 
   // Output buffer name
-  std::string const m_bufferName;
+  Gaudi::Property<std::string> m_bufferName{this, "BufferName", "Output"};
+
+  // Do output checksum
+  Gaudi::Property<bool> m_checksum {this, "Checksum", false};
 
   // Process name
-  std::string const m_processName;
+  std::string m_processName;
 
   // partition ID
   unsigned m_partitionID = 0;
@@ -61,4 +67,16 @@ private:
   BMID m_bmID = MBM_INV_DESC;
 
   bool m_cancelled = false;
+
+  std::mutex m_mutex;
+  size_t m_id = 0;
+  std::list<size_t> m_freeIDs;
+  std::unordered_map<size_t, size_t> m_eventCounts;
+
+  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_burstsOutput;
+  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsOutput;
+  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbOutput;
+
+
 };
+}
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 943ebc59b..ef932aefa 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -275,6 +275,22 @@ StatusCode MEPProvider::initialize()
     info() << endmsg;
   }
 
+  auto eventsSvc = dynamic_cast<Service*>(service<IService>("Service/Events", true).get());
+  if (!eventsSvc) {
+    error() << "Failed to obtain Events service for monitoring" << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  auto burstsSvc = dynamic_cast<Service*>(service<IService>("Service/Bursts", true).get());
+  if (!eventsSvc) {
+    error() << "Failed to obtain Bursts service for monitoring" << endmsg;
+    return StatusCode::FAILURE;
+  }
+
+  m_mepsInput = std::make_unique<Gaudi::Accumulators::Counter<>>(burstsSvc, "IN");
+  m_eventsInput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "IN");
+  m_mbInput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "MB_IN");
+
   #ifdef HAVE_MPI
   if (!m_buffer_numa.value().empty() && m_buffer_numa.value().size() != n_buffers()) {
     error() << "Buffer NUMA domains must be specified for all buffers" << endmsg;
@@ -979,8 +995,14 @@ void MEPProvider::mep_read()
         auto& status = m_buffer_status[i_buffer];
         assert(status.work_counter == 0);
 
+        auto const n_events = (to_read ? to_publish : size_t {slice.packing_factor});
+
+        (*m_mepsInput) += 1;
+        (*m_eventsInput) += n_events;
+        (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
+
         if (!eof && to_publish != 0) {
-          set_intervals(status.intervals, to_read ? to_publish : size_t {slice.packing_factor});
+          set_intervals(status.intervals, n_events);
         }
         else {
           // We didn't read anything, so free the buffer we got again
@@ -1175,6 +1197,12 @@ void MEPProvider::mpi_read()
     auto& [meps_received, bytes_received] = data_received[receiver];
     bytes_received += mep_size;
     meps_received += 1;
+
+    (*m_mepsInput) += 1;
+    (*m_eventsInput) += slice.packing_factor;
+    (*m_mbInput) += (2 * mep_size + 1) / (2 * 1024 * 1024);
+
+
     if (t.get_elapsed_time() >= reporting_period) {
       const auto seconds = t.get_elapsed_time();
       auto total_rate = 0.;
@@ -1331,13 +1359,16 @@ void MEPProvider::bm_read(const std::string& buffer_name)
       }
 
       // info() << "Got MEP " << i_buffer << endmsg;
-
       slice.mep = reinterpret_cast<EB::MEP const*>(ev_data);
       slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep->bytes()};
       slice.slice_size = static_cast<size_t>(ev_len);
       auto const* mfp = slice.mep->at(0);
       slice.packing_factor = mfp->header.n_banks;
 
+      (*m_mepsInput) += 1;
+      (*m_eventsInput) += slice.packing_factor;
+      (*m_mbInput) += (2 * slice.slice_size + 1) / (2 * 1024 * 1024);
+
       if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
     } else if (sc == MBM_REQ_CANCEL) {
       std::unique_lock<std::mutex> lock {m_buffer_mutex};
-- 
GitLab


From e15b681580832456fada86cf75df44008a4dba65 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Sat, 30 Oct 2021 15:09:45 +0200
Subject: [PATCH 068/120] Additional counter for MBs input and output

---
 AllenOnline/options/Allen.py    | 3 +--
 AllenOnline/src/MBMOutput.h     | 4 ++--
 AllenOnline/src/MEPProvider.cpp | 2 +-
 3 files changed, 4 insertions(+), 5 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 4dd3705eb..487ed3cbf 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -126,7 +126,7 @@ else:
     mep_provider.Connections = sorted([os.path.join(mep_dir, mep_file)
                                        for mep_file in os.listdir(mep_dir)
                                        if mep_file.endswith('.mep')])
-    mep_provider.LoopOnMEPs = True
+    mep_provider.LoopOnMEPs = False
 
 monSvc = MonitorSvc('MonitorSvc')
 monSvc.ExpandNameInfix = '<proc>/'
@@ -135,7 +135,6 @@ monSvc.UniqueServiceNames = True
 monSvc.UseDStoreNames = True
 monSvc.PartitionName = partition
 monSvc.DimUpdateInterval = 5
-monSvc.OutputLevel = 2
 
 setup_allen_non_event_data_service()
 
diff --git a/AllenOnline/src/MBMOutput.h b/AllenOnline/src/MBMOutput.h
index 454d9a44a..c49985f84 100644
--- a/AllenOnline/src/MBMOutput.h
+++ b/AllenOnline/src/MBMOutput.h
@@ -77,6 +77,6 @@ private:
   std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsOutput;
   std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbOutput;
 
-
 };
-}
+
+} // namespace Allen
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index ef932aefa..d9fbcb669 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -1367,7 +1367,7 @@ void MEPProvider::bm_read(const std::string& buffer_name)
 
       (*m_mepsInput) += 1;
       (*m_eventsInput) += slice.packing_factor;
-      (*m_mbInput) += (2 * slice.slice_size + 1) / (2 * 1024 * 1024);
+      (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
 
       if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
     } else if (sc == MBM_REQ_CANCEL) {
-- 
GitLab


From 1a88981676bdf870d2ab24b4844507c25e5e98d7 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 3 Dec 2021 23:57:33 +0100
Subject: [PATCH 069/120] Add trivial service for monitoring purposes and use
 it.

---
 AllenOnline/CMakeLists.txt      |  2 ++
 AllenOnline/src/IOMonitor.cpp   | 14 ++++++++++++++
 AllenOnline/src/MBMOutput.cpp   |  4 ++--
 AllenOnline/src/MEPProvider.cpp | 16 +++++++++-------
 4 files changed, 27 insertions(+), 9 deletions(-)
 create mode 100644 AllenOnline/src/IOMonitor.cpp

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 46aaa8dd1..461bb7251 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -32,6 +32,7 @@ gaudi_add_module(AllenOnlineComp
         src/AllenConfiguration.cpp
         src/MEPProvider.cpp
         src/MBMOutput.cpp
+        src/IOMonitor.cpp
     LINK
         Gaudi::GaudiKernel
         Online::dim
@@ -48,6 +49,7 @@ gaudi_add_module(AllenOnlineComp
         LHCb::DAQEventLib
         LHCb::DAQKernelLib
         Gaudi::GaudiAlgLib
+        cppgsl::cppgsl
         AllenOnlineLib
         rt)
 
diff --git a/AllenOnline/src/IOMonitor.cpp b/AllenOnline/src/IOMonitor.cpp
new file mode 100644
index 000000000..b22d60cf1
--- /dev/null
+++ b/AllenOnline/src/IOMonitor.cpp
@@ -0,0 +1,14 @@
+/*****************************************************************************\
+* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
+\*****************************************************************************/
+#include <GaudiKernel/Service.h>
+
+namespace Allen {
+  class IOMonitor : public Service {
+
+    using Service::Service;
+
+  };
+}
+
+DECLARE_COMPONENT_WITH_ID(Allen::IOMonitor, "AllenIOMon")
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index b8edef2c0..3ed0687ec 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -94,13 +94,13 @@ StatusCode Allen::MBMOutput::initialize()
     connection += partition.empty() ? hexID.str() : partition;
   }
 
-  auto eventsSvc = dynamic_cast<Service*>(service<IService>("Service/Events", true).get());
+  auto eventsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Events", true).get());
   if (!eventsSvc) {
     error() << "Failed to obtain Events service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
 
-  auto burstsSvc = dynamic_cast<Service*>(service<IService>("Service/Bursts", true).get());
+  auto burstsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Bursts", true).get());
   if (!eventsSvc) {
     error() << "Failed to obtain Bursts service for monitoring" << endmsg;
     return StatusCode::FAILURE;
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index d9fbcb669..85d36c3b2 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -257,7 +257,7 @@ StatusCode MEPProvider::initialize()
     m_buffer_status[i].index = i;
   }
 
-  auto config = service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
+  auto config = service("AllenConfiguration/AllenConfiguration", false).as<AllenConfiguration>();
   if (!config) {
     error() << "Failed to retrieve AllenConfiguration." << endmsg;
     return StatusCode::FAILURE;
@@ -275,13 +275,13 @@ StatusCode MEPProvider::initialize()
     info() << endmsg;
   }
 
-  auto eventsSvc = dynamic_cast<Service*>(service<IService>("Service/Events", true).get());
+  auto eventsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Events", true).get());
   if (!eventsSvc) {
     error() << "Failed to obtain Events service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
 
-  auto burstsSvc = dynamic_cast<Service*>(service<IService>("Service/Bursts", true).get());
+  auto burstsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Bursts", true).get());
   if (!eventsSvc) {
     error() << "Failed to obtain Bursts service for monitoring" << endmsg;
     return StatusCode::FAILURE;
@@ -997,11 +997,13 @@ void MEPProvider::mep_read()
 
         auto const n_events = (to_read ? to_publish : size_t {slice.packing_factor});
 
-        (*m_mepsInput) += 1;
-        (*m_eventsInput) += n_events;
-        (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
-
         if (!eof && to_publish != 0) {
+          // Monitor this MEP
+          (*m_mepsInput) += 1;
+          (*m_eventsInput) += n_events;
+          (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
+
+          // Set intervals for offset calculation/transposition
           set_intervals(status.intervals, n_events);
         }
         else {
-- 
GitLab


From 575c6a357066bcc6fe6c00e7986119c625ee372f Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 6 Dec 2021 12:39:13 +0100
Subject: [PATCH 070/120] Add declare component macro for MBMOutput

---
 AllenOnline/src/MBMOutput.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 3ed0687ec..8948a63fb 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -15,6 +15,8 @@
 
 #include "MBMOutput.h"
 
+DECLARE_COMPONENT(Allen::MBMOutput)
+
 Allen::MBMOutput::MBMOutput(std::string name, ISvcLocator* loc)
   : Service {name, loc}
 {
-- 
GitLab


From d9aa0680bb6adf9db26ef12c0bf14121c41c3b63 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 7 Dec 2021 09:58:13 +0100
Subject: [PATCH 071/120] Fix propagation of failure to open file

---
 AllenOnline/src/MEPProvider.cpp | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 85d36c3b2..faa6e88dc 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -785,7 +785,7 @@ bool MEPProvider::open_file() const
     if (m_input) m_input->close();
 
     m_input = LHCb::StreamDescriptor::bind(*m_current);
-    if (m_input->ioDesc != 0) {
+    if (m_input->ioDesc > 0) {
       info() << "Opened " << *m_current << endmsg;
       good = true;
     }
@@ -858,6 +858,7 @@ void MEPProvider::mep_read()
   auto read_error = [this] {
     m_read_error = true;
     m_transpose_cond.notify_all();
+    m_transposed_cond.notify_one();
   };
 
   auto to_read = this->n_events();
-- 
GitLab


From 5ba1f70126a2ab1acf916319134cd9123d168556 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 22 Mar 2022 16:06:57 +0100
Subject: [PATCH 072/120] Follow changes in Online

---
 AllenOnline/CMakeLists.txt | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 461bb7251..9be63d6e4 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -20,7 +20,6 @@ gaudi_add_library(AllenOnlineLib
     LINK PUBLIC
         Gaudi::GaudiKernel
         cppgsl::cppgsl
-        Online::OnlineKernel
         Online::EventBuildingLib
         Allen::AllenLib
 )
@@ -43,7 +42,6 @@ gaudi_add_module(AllenOnlineComp
         Gaudi::Parsers
         Online::EventBuildingLib
         LHCb::MDFLib
-        Online::OnlineKernel
         Allen::AllenLib
         Allen::BinaryDumpers
         LHCb::DAQEventLib
@@ -51,7 +49,8 @@ gaudi_add_module(AllenOnlineComp
         Gaudi::GaudiAlgLib
         cppgsl::cppgsl
         AllenOnlineLib
-        rt)
+        rt
+)
 
 # MPI
 if (MPI_FOUND AND hwloc_FOUND)
@@ -77,7 +76,8 @@ if (MPI_FOUND AND hwloc_FOUND)
         Boost::program_options
         MPI::MPI_CXX
         PkgConfig::hwloc
-        ${open_pal_lib})
+        ${open_pal_lib}
+   )
    target_include_directories(allen_mpi_send PRIVATE ${MPI_CXX_INCLUDE_DIRS})
    target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
 endif()
@@ -88,7 +88,8 @@ gaudi_add_executable(allen_read_mep
     LINK
       AllenOnline
       LHCb::MDFLib
-      Boost::program_options)
+      Boost::program_options
+)
 
 gaudi_add_executable(allen_test_host_register
     SOURCES
@@ -98,4 +99,5 @@ gaudi_add_executable(allen_test_host_register
       Online::OnlineBase
       Allen::Backend
       LHCb::ZMQLib
-    TEST)
+    TEST
+)
-- 
GitLab


From e411f3ad7789cbf66a734b5aa8bd8836d36b7dde Mon Sep 17 00:00:00 2001
From: rta <rta@tdeb03.lbdaq.cern.ch>
Date: Tue, 22 Mar 2022 17:06:10 +0100
Subject: [PATCH 073/120] Fix configuration and add some useful output messages

---
 AllenOnline/options/Allen.py  | 45 +++++++++++++++++++----------------
 AllenOnline/src/MBMOutput.cpp |  7 ++++--
 2 files changed, 30 insertions(+), 22 deletions(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 487ed3cbf..9ada97805 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -10,7 +10,9 @@
 # or submit itself to any jurisdiction.                                       #
 ###############################################################################
 import os
+import re
 import json
+from Gaudi.Configuration import importOptions, allConfigurables
 from Configurables import LHCbApp, CondDB, ApplicationMgr
 from Configurables import DumpUTGeometry, DumpFTGeometry, DumpMuonTable
 from Configurables import DumpMuonGeometry, DumpVPGeometry, AllenUpdater
@@ -23,6 +25,8 @@ from Allen.config import setup_allen_non_event_data_service
 
 try:
     import OnlineEnvBase as OnlineEnv
+    importOptions("$MBM_SETUP_OPTIONS")
+    mbm_setup = allConfigurables['OnlineEnv']
     run_online = True
     output_level = OnlineEnv.OutputLevel
     partition = OnlineEnv.PartitionName
@@ -41,6 +45,10 @@ app = LHCbApp(
     DDDBtag="dddb-20210218",
     CondDBtag="sim-20201218-vc-md100")
 
+setup_allen_non_event_data_service()
+
+appMgr = ApplicationMgr()
+
 # Upgrade DBs
 CondDB().Upgrade = True
 
@@ -66,11 +74,11 @@ online_conf.OutputLevel = 3
 online_conf.IOOutputLevel = output_level
 
 if run_online:
-    import _fifo_log
+    import fifo_log
     online_conf.logDeviceType = 'fifo'
     online_conf.logDeviceFormat = '%-8LEVEL %-24SOURCE'
-    _fifo_log.logger_set_tag(partition)
-    _fifo_log.logger_start()
+    fifo_log.logger_set_tag(partition)
+    fifo_log.logger_start()
 
 allen_conf = AllenConfiguration()
 allen_conf.StopTimeout = 5.
@@ -91,9 +99,10 @@ allen_conf.EnableRunChanges = True
 if run_online:
     from Configurables import Allen__MBMOutput as MBMOutput
     output_svc = MBMOutput()
+    output_svc.OutputLevel = 3
     output_svc.Checksum = True
-    output_svc.BufferName = "Output"
-    app.ExtSvc += [output_svc]
+    output_svc.BufferName = mbm_setup.Allen_Output
+    appMgr.ExtSvc += [output_svc]
 elif integration_test:
     allen_conf.Output = "tcp://192.168.1.101:35000"
 else:
@@ -112,7 +121,7 @@ mep_provider.ThreadPerBuffer = True
 if run_online:
     mep_provider.Source = "MBM"
     mep_provider.MBMComMethod = "FIFO"
-    mep_provider.Connections = ["Events_0", "Events_1"]
+    mep_provider.Connections = mbm_setup.BU_Buffers #  ["Events_0", "Events_1"]
     mep_provider.Requests = [
         'EvType=1;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
     ]
@@ -136,27 +145,23 @@ monSvc.UseDStoreNames = True
 monSvc.PartitionName = partition
 monSvc.DimUpdateInterval = 5
 
-setup_allen_non_event_data_service()
-
-app = ApplicationMgr()
-
-app.MessageSvcType = 'MessageSvc'
-app.OutputLevel = output_level
+appMgr.MessageSvcType = 'MessageSvc'
+appMgr.OutputLevel = output_level
 messageSvc = MessageSvc('MessageSvc')
 messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
 messageSvc.OutputLevel = output_level
 
 
 # Add the services that will produce the non-event-data
-app.ExtSvc = [
-    monSvc,
-    "OnlMonitorSink"
-] + app.ExtSvc
+# appMgr.ExtSvc = [
+#     monSvc,
+#     "OnlMonitorSink"
+# ] + appMgr.ExtSvc
 
 # Some extra stuff for timing table
-ApplicationMgr().EvtSel = "NONE"
-ApplicationMgr().ExtSvc += [
-    'ToolSvc', 'AuditorSvc', 'AllenConfiguration',
+appMgr.EvtSel = "NONE"
+appMgr.ExtSvc = [
+    'ToolSvc', 'AuditorSvc', allen_conf,
     'Online::Configuration/Application', 'ZeroMQSvc',
     'MEPProvider'
-]
+] + appMgr.ExtSvc
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 8948a63fb..3725bb205 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -88,14 +88,17 @@ StatusCode Allen::MBMOutput::initialize()
 
   m_partitionID = m_allenConfig->partitionID.value();
   std::string connection = m_bufferName.value();
-  if (m_allenConfig->partitionBuffers.value()) {
+  auto const partition = m_allenConfig->partition.value();
+  bool partitionBuffers = m_allenConfig->partitionBuffers.value();
+  if (partitionBuffers) {
     connection += "_";
-    auto const partition = m_allenConfig->partition.value();
     std::stringstream hexID;
     hexID << std::hex << m_partitionID;
     connection += partition.empty() ? hexID.str() : partition;
   }
 
+  info() << "Connect to MBM: " << connection << endmsg;
+
   auto eventsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Events", true).get());
   if (!eventsSvc) {
     error() << "Failed to obtain Events service for monitoring" << endmsg;
-- 
GitLab


From 6a8937bfea5205a2aa53028a036433b3d7743cb7 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 28 Mar 2022 17:03:36 +0200
Subject: [PATCH 074/120] Fix rebase and refactor file layout a bit

---
 AllenOnline/CMakeLists.txt                    | 23 +++++++++----------
 AllenOnline/application/read_mep.cpp          |  2 +-
 .../include/{ => AllenOnline}/ReadMEP.h       |  0
 .../{ => include}/AllenOnline/TransposeMEP.h  |  0
 AllenOnline/src/AllenApplication.cpp          |  4 ++--
 AllenOnline/src/MEPProvider.cpp               |  5 ++--
 .../{AllenOnline => src}/MEPProvider.h        |  2 +-
 AllenOnline/{include => src}/MPIConfig.h      |  0
 AllenOnline/src/ReadMEP.cpp                   |  2 +-
 CMakeLists.txt                                |  1 +
 10 files changed, 19 insertions(+), 20 deletions(-)
 rename AllenOnline/include/{ => AllenOnline}/ReadMEP.h (100%)
 rename AllenOnline/{ => include}/AllenOnline/TransposeMEP.h (100%)
 rename AllenOnline/{AllenOnline => src}/MEPProvider.h (99%)
 rename AllenOnline/{include => src}/MPIConfig.h (100%)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 9be63d6e4..f0884df9e 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -33,22 +33,21 @@ gaudi_add_module(AllenOnlineComp
         src/MBMOutput.cpp
         src/IOMonitor.cpp
     LINK
-        Gaudi::GaudiKernel
-        Online::dim
-        Online::OnlineBase
-        Online::GaudiOnline
-        LHCb::ZMQLib
-        Online::RPC
-        Gaudi::Parsers
-        Online::EventBuildingLib
-        LHCb::MDFLib
         Allen::AllenLib
         Allen::BinaryDumpers
+        AllenOnlineLib
+        Gaudi::GaudiAlgLib
+        Gaudi::GaudiKernel
         LHCb::DAQEventLib
         LHCb::DAQKernelLib
-        Gaudi::GaudiAlgLib
+        LHCb::MDFLib
+        LHCb::ZMQLib
+        Online::EventBuildingLib
+        Online::GaudiOnline
+        Online::OnlineBase
+        Online::RPC
+        Online::dim
         cppgsl::cppgsl
-        AllenOnlineLib
         rt
 )
 
@@ -86,7 +85,7 @@ gaudi_add_executable(allen_read_mep
     SOURCES
       application/read_mep.cpp
     LINK
-      AllenOnline
+      AllenOnlineLib
       LHCb::MDFLib
       Boost::program_options
 )
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 697cf563a..4ce0c7fb6 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -27,7 +27,7 @@
 #include <EventBuilding/MFP_tools.hpp>
 
 #include <AllenOnline/TransposeMEP.h>
-#include <ReadMEP.h>
+#include <AllenOnline/ReadMEP.h>
 
 using namespace std;
 
diff --git a/AllenOnline/include/ReadMEP.h b/AllenOnline/include/AllenOnline/ReadMEP.h
similarity index 100%
rename from AllenOnline/include/ReadMEP.h
rename to AllenOnline/include/AllenOnline/ReadMEP.h
diff --git a/AllenOnline/AllenOnline/TransposeMEP.h b/AllenOnline/include/AllenOnline/TransposeMEP.h
similarity index 100%
rename from AllenOnline/AllenOnline/TransposeMEP.h
rename to AllenOnline/include/AllenOnline/TransposeMEP.h
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index d4c10513f..b03f31413 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -40,10 +40,10 @@
 #include <Allen/Allen.h>
 #include <Allen/Provider.h>
 
-#include <AllenOnline/MEPProvider.h>
+#include "MEPProvider.h"
 
 #ifdef HAVE_MPI
-#include <MPIConfig.h>
+#include "MPIConfig.h"
 #endif
 
 #include "AllenConfiguration.h"
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index faa6e88dc..e7c104bce 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -20,13 +20,12 @@
 
 #include <MBM/Requirement.h>
 
-
-#include "AllenOnline/MEPProvider.h"
 #include "AllenOnline/TransposeMEP.h"
+#include "AllenOnline/ReadMEP.h"
 
+#include "MEPProvider.h"
 #include "AllenConfiguration.h"
 
-#include "ReadMEP.h"
 
 #ifdef HAVE_MPI
 #include "MPIConfig.h"
diff --git a/AllenOnline/AllenOnline/MEPProvider.h b/AllenOnline/src/MEPProvider.h
similarity index 99%
rename from AllenOnline/AllenOnline/MEPProvider.h
rename to AllenOnline/src/MEPProvider.h
index f32b6b497..01ea205d7 100644
--- a/AllenOnline/AllenOnline/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -33,7 +33,7 @@
 
 #include <MDF/StreamDescriptor.h>
 
-#include "TransposeMEP.h"
+#include <AllenOnline/TransposeMEP.h>
 
 #ifdef HAVE_MPI
 #include <hwloc.h>
diff --git a/AllenOnline/include/MPIConfig.h b/AllenOnline/src/MPIConfig.h
similarity index 100%
rename from AllenOnline/include/MPIConfig.h
rename to AllenOnline/src/MPIConfig.h
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index c6691471d..38739ced1 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -20,7 +20,7 @@
 #include <MDF/PosixIO.h>
 #include <Allen/BankTypes.h>
 
-#include "ReadMEP.h"
+#include <AllenOnline/ReadMEP.h>
 
 namespace {
   using std::cerr;
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 394a9e322..82a18d2fe 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -27,6 +27,7 @@ include(MooreOnlineDependencies)
 
 # -- Subdirectories
 lhcb_add_subdirectories(
+    AllenOnline
     MooreOnlineConf
     MooreScripts
     Online/Converters
-- 
GitLab


From 024854dbf7ecc3d67af6510920d86bdaf4c23946 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 29 Mar 2022 11:01:38 +0200
Subject: [PATCH 075/120] Fix build if MPI and hwloc are present

---
 AllenOnline/CMakeLists.txt                    | 58 +++++++++----------
 AllenOnline/application/MPISend.cpp           |  9 ++-
 .../{src => include/AllenOnline}/MPIConfig.h  |  0
 AllenOnline/src/AllenApplication.cpp          |  2 +-
 AllenOnline/src/MEPProvider.cpp               | 12 ++--
 5 files changed, 43 insertions(+), 38 deletions(-)
 rename AllenOnline/{src => include/AllenOnline}/MPIConfig.h (100%)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index f0884df9e..e6f797b60 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -51,36 +51,6 @@ gaudi_add_module(AllenOnlineComp
         rt
 )
 
-# MPI
-if (MPI_FOUND AND hwloc_FOUND)
-  message(STATUS "Found MPI ${MPI_CXX_VERSION}: ${MPI_CXX_COMPILER}")
-
-  # open-pal library is also needed...
-  set(MPI_LIBDIRS)
-  foreach(mpi_lib ${MPI_CXX_LIBRARIES})
-    get_filename_component(_lib_dir ${mpi_lib} DIRECTORY)
-    list(APPEND MPI_LIBDIRS ${_lib_dir})
-  endforeach()
-  find_library(open_pal_lib open-pal PATHS ${MPI_LIBDIRS} REQUIRED)
-
-  target_link_libraries(AllenOnlineComp MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
-  target_include_directories(AllenOnlineComp PRIVATE SYSTEM ${MPI_CXX_INCLUDE_DIRS})
-  target_compile_definitions(AllenOnlineComp PRIVATE HAVE_MPI)
-
-  gaudi_add_executable(allen_mpi_send
-      SOURCES
-        application/MPISend.cpp
-      LINK
-        AllenOnline LHCb::MDFLib
-        Boost::program_options
-        MPI::MPI_CXX
-        PkgConfig::hwloc
-        ${open_pal_lib}
-   )
-   target_include_directories(allen_mpi_send PRIVATE ${MPI_CXX_INCLUDE_DIRS})
-   target_compile_definitions(AllenMPISend PRIVATE HAVE_MPI)
-endif()
-
 gaudi_add_executable(allen_read_mep
     SOURCES
       application/read_mep.cpp
@@ -100,3 +70,31 @@ gaudi_add_executable(allen_test_host_register
       LHCb::ZMQLib
     TEST
 )
+
+gaudi_add_executable(allen_mpi_send
+   SOURCES
+     application/MPISend.cpp
+   LINK
+     AllenOnlineLib
+     LHCb::MDFLib
+     Boost::program_options
+)
+
+# MPI
+if (TARGET MPI::MPI_CXX AND TARGET PkgConfig::hwloc)
+  message(STATUS "Found MPI ${MPI_CXX_VERSION}: ${MPI_CXX_COMPILER}")
+
+  # open-pal library is also needed...
+  set(MPI_LIBDIRS)
+  foreach(mpi_lib ${MPI_CXX_LIBRARIES})
+    get_filename_component(_lib_dir ${mpi_lib} DIRECTORY)
+    list(APPEND MPI_LIBDIRS ${_lib_dir})
+  endforeach()
+  find_library(open_pal_lib open-pal PATHS ${MPI_LIBDIRS} REQUIRED)
+
+  foreach(component AllenOnlineComp allen_mpi_send)
+    target_link_libraries(${component} PRIVATE MPI::MPI_CXX PkgConfig::hwloc ${open_pal_lib})
+    target_include_directories(${component} PRIVATE SYSTEM ${MPI_CXX_INCLUDE_DIRS})
+    target_compile_definitions(${component} PRIVATE HAVE_MPI)
+  endforeach()
+endif()
diff --git a/AllenOnline/application/MPISend.cpp b/AllenOnline/application/MPISend.cpp
index c125981ce..97d7bc24a 100644
--- a/AllenOnline/application/MPISend.cpp
+++ b/AllenOnline/application/MPISend.cpp
@@ -15,8 +15,8 @@
 #include <boost/program_options.hpp>
 #include <boost/algorithm/string.hpp>
 
-#include <ReadMEP.h>
-#include <MPIConfig.h>
+#include <AllenOnline/ReadMEP.h>
+#include <AllenOnline/MPIConfig.h>
 
 namespace MPI {
   int rank;
@@ -74,6 +74,7 @@ int main(int argc, char* argv[]) {
     return -1;
   }
 
+#if defined(HAVE_MPI)
   std::vector<std::string> connections;
   ba::split(connections, mep_input, ba::is_any_of(","));
 
@@ -236,4 +237,8 @@ send:
 
   MPI_Finalize();
   return 0;
+#else
+  std::cerr << "Not built with MPI support\n";
+  return 1;
+#endif
 }
diff --git a/AllenOnline/src/MPIConfig.h b/AllenOnline/include/AllenOnline/MPIConfig.h
similarity index 100%
rename from AllenOnline/src/MPIConfig.h
rename to AllenOnline/include/AllenOnline/MPIConfig.h
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index b03f31413..4a8f9ba29 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -43,7 +43,7 @@
 #include "MEPProvider.h"
 
 #ifdef HAVE_MPI
-#include "MPIConfig.h"
+#include <AllenOnline/MPIConfig.h>
 #endif
 
 #include "AllenConfiguration.h"
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index e7c104bce..6908cb5c2 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -20,15 +20,15 @@
 
 #include <MBM/Requirement.h>
 
-#include "AllenOnline/TransposeMEP.h"
-#include "AllenOnline/ReadMEP.h"
+#include <AllenOnline/TransposeMEP.h>
+#include <AllenOnline/ReadMEP.h>
 
 #include "MEPProvider.h"
 #include "AllenConfiguration.h"
 
 
 #ifdef HAVE_MPI
-#include "MPIConfig.h"
+#include <AllenOnline/MPIConfig.h>
 #include <hwloc.h>
 #endif
 
@@ -1579,8 +1579,8 @@ void MEPProvider::transpose(int thread_id)
   }
 }
 
-StatusCode MEPProvider::numa_membind(char const* mem, size_t size, int const numa_node) const {
 #ifdef HAVE_MPI
+StatusCode MEPProvider::numa_membind(char const* mem, size_t size, int const numa_node) const {
   auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
   auto s = hwloc_set_area_membind(
     m_topology, mem, size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET | HWLOC_MEMBIND_STRICT | HWLOC_MEMBIND_MIGRATE);
@@ -1592,7 +1592,9 @@ StatusCode MEPProvider::numa_membind(char const* mem, size_t size, int const num
   else {
     return StatusCode::SUCCESS;
   }
+}
 #else
+StatusCode MEPProvider::numa_membind(char const*, size_t, int const) const {
   return StatusCode::FAILURE;
-#endif
 }
+#endif
-- 
GitLab


From 0c0b6be30ddb0082ad406a39ca3d86ea127ea558 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 1 Mar 2022 16:03:52 +0100
Subject: [PATCH 076/120] Follow changes in Allen to use fragment sizes

---
 AllenOnline/src/MEPProvider.cpp  | 32 +++++++++---------
 AllenOnline/src/TransposeMEP.cpp | 56 ++++++++++++++++++--------------
 2 files changed, 48 insertions(+), 40 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 6908cb5c2..82d011783 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -69,22 +69,23 @@ BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) cons
   auto ib = to_integral(bank_type);
   assert(static_cast<size_t>(ib) < m_slices.size());
   assert(slice_index < m_slices[ib].size());
-  auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
+  auto const& slice = m_slices[ib][slice_index];
+  // auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
 
   BanksAndOffsets bno;
-  auto& spans = std::get<0>(bno);
-  spans.reserve(banks.size());
-  for (auto s : banks) {
+  auto& spans = bno.fragments;
+  spans.reserve(slice.fragments.size());
+  for (auto s : slice.fragments) {
     spans.emplace_back(s);
   }
 
-  std::get<1>(bno) = m_transpose_mep.value() ? offsets[offsets_size - 1] : data_size;
-  std::get<2>(bno) = offsets;
-  std::get<3>(bno) = m_banks_version[ib];
+  bno.fragments_mem_size = m_transpose_mep.value() ? slice.offsets[slice.n_offsets - 1] : slice.fragments_mem_size;
+  bno.offsets = slice.offsets;
+  bno.version = m_banks_version[ib];
 
   if (msgLevel(MSG::VERBOSE)) {
     verbose() << "slice " << std::setw(3) << slice_index << " bank type " << ib
-              << " size " << std::setw(12) << std::get<1>(bno) << endmsg;
+              << " size " << std::setw(12) << bno.fragments_mem_size << endmsg;
   }
 
   return bno;
@@ -719,10 +720,13 @@ bool MEPProvider::allocate_storage(size_t i_read)
 
   // Allocate slice memory that will contain transposed banks ready
   // for processing by the Allen kernels
-  auto size_fun = [this, eps](BankTypes bank_type) -> std::tuple<size_t, size_t> {
+  auto size_fun = [this, eps](BankTypes bank_type) -> std::tuple<size_t, size_t, size_t> {
     auto it = BankSizes.find(bank_type);
     auto const sd_index = to_integral(bank_type);
 
+    auto aps = eps < 100 ? 100 : eps;
+    auto n_sizes = aps * ((Allen::max_fragments + 1) / 2 + 1);
+
     if (it == end(BankSizes)) {
       throw std::out_of_range {std::string {"Bank type "} + bank_name(bank_type) + " has no known size"};
     }
@@ -730,25 +734,21 @@ bool MEPProvider::allocate_storage(size_t i_read)
     if (!m_transpose_mep.value()) {
       auto n_blocks = m_mfp_count[sd_index];
       // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
-      return {0, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
+      return {0, n_sizes, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
     }
     else {
-      auto aps = eps < 100 ? 100 : eps;
       // Lookup LHCb bank type corresponding to Allen bank type
 
       // When events are transposed from the read buffer into
       // the per-rawbank-type slices, a check is made each time
       // to see if there is enough space available in a slice.
       // To avoid having to read every event twice to get the
-      // size of all the banks, the size of the entire event is
-      // used for the check - 65 kB on average. To avoid
-      // problems for banks with very low average size like the
-      // ODIN bank - 0.1 kB, a fixed amount is also added.
+      // size of all the banks.
       auto n_bytes = std::lround(
         ((1 + m_mfp_count[sd_index]) * sizeof(uint32_t) + it->second * kB) * aps *
           bank_size_fudge_factor +
         2 * MB); // FIXME for the banks_count
-      return {n_bytes, eps};
+      return {n_bytes, n_sizes, eps};
     }
   };
   m_slices = allocate_slices(m_nslices, m_bank_types, size_fun);
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index a4e1e672a..fb7ff214e 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -103,7 +103,7 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
     unsigned    run_number     = 0;
 
     if ( bank_types.count( allen_type ) ) {
-      auto& event_offsets = std::get<2>( slices[sd_index][slice_index] );
+      auto& event_offsets = slices[sd_index][slice_index].offsets;
 
       for ( size_t i = event_start; i < event_end; ++i ) {
         // First check for run changes in ODIN banks
@@ -137,17 +137,17 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
   size_t n_frag = ( event_end - event_start );
   for ( auto allen_type : bank_types ) {
     auto const sd_index = to_integral(allen_type);
-    auto& [slice, slice_size, event_offsets, offsets_size] = slices[sd_index][slice_index];
-    event_offsets[0]                                       = 0;
-    auto preamble_words                                    = 2 + mfp_count[sd_index];
+    auto& slice = slices[sd_index][slice_index];
+    slice.offsets[0]    = 0;
+    auto preamble_words = 2 + mfp_count[sd_index];
     for ( size_t i = 1; i <= ( event_end - event_start ) && i <= n_frag; ++i ) {
 
       // Allen raw bank format has the number of banks and the bank
       // offsets in a preamble
-      event_offsets[i] += preamble_words * sizeof( uint32_t ) + event_offsets[i - 1];
+      slice.offsets[i] += preamble_words * sizeof( uint32_t ) + slice.offsets[i - 1];
 
       // Check for sufficient space
-      if ( event_offsets[i] > slice_size ) {
+      if ( slice.offsets[i] > slice.fragments_mem_size ) {
         n_frag = i - 1;
         break;
       }
@@ -157,8 +157,7 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
   // Set offsets_size here to make sure it's consistent with the max
   for ( auto allen_type : bank_types ) {
     auto const sd_index = to_integral(allen_type);
-    auto& offsets_size = std::get<3>( slices[sd_index][slice_index] );
-    offsets_size       = n_frag + 1;
+    slices[sd_index][slice_index].n_offsets = n_frag + 1;
   }
   return n_frag;
 }
@@ -217,28 +216,37 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
     }
 
     if ( bank_types.count( allen_type ) ) {
-      auto& [spans, data_size, event_offsets, offsets_size] = slices[sd_index][slice_index];
+      auto& slice = slices[sd_index][slice_index];
+      auto& fragments = slice.fragments;
+      auto& offsets = slice.offsets;
+      auto& offsets_size = slice.n_offsets;
 
       // Calculate block offset and size
       size_t interval_offset = 0, interval_size = 0;
-      for ( size_t i = 0; i < event_start; ++i ) { interval_offset += bank_sizes[i]; }
-      for ( size_t i = event_start; i < event_end; ++i ) { interval_size += bank_sizes[i]; }
+      for ( size_t i = 0; i < event_start; ++i ) {
+        auto const padded_size = bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
+        interval_offset += padded_size;
+      }
+      for ( size_t i = event_start; i < event_end; ++i ) {
+        auto const padded_size = bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
+        interval_size += padded_size;
+      }
 
       // Calculate offsets
       if ( allen_type != prev_type ) {
-        event_offsets[0]            = mfp_count[sd_index];
-        event_offsets[1]            = event_end - event_start;
-        event_offsets[2 + n_blocks] = 0;
-        offset_index                = 0;
-        prev_type                   = allen_type;
+        offsets[0]            = mfp_count[sd_index];
+        offsets[1]            = event_end - event_start;
+        offsets[2 + n_blocks] = 0;
+        offset_index          = 0;
+        prev_type             = allen_type;
       }
 
       // Store source ID
-      event_offsets[2 + offset_index] = source_id;
+      offsets[2 + offset_index] = source_id;
 
       // Initialize the first offsets using the block sizes,
       if ( offset_index < mfp_count[sd_index] - 1 ) {
-        event_offsets[2 + n_blocks + offset_index + 1] = event_offsets[2 + n_blocks + offset_index] + interval_size;
+        offsets[2 + n_blocks + offset_index + 1] = offsets[2 + n_blocks + offset_index] + interval_size;
       }
 
       // Fill fragment offsets
@@ -246,14 +254,14 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       for ( size_t i = event_start; i < event_end; ++i ) {
         idx               = i - event_start + 1;
         oi                = 2 + n_blocks * ( 1 + idx ) + offset_index;
-        event_offsets[oi] = event_offsets[oi - n_blocks] + bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
+        offsets[oi] = offsets[oi - n_blocks] + bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
       }
       // Update offsets_size
       offsets_size = oi;
 
       // Store block span for this interval
-      spans.emplace_back( const_cast<char*>( block.payload ) + interval_offset, interval_size );
-      data_size += interval_size;
+      fragments.emplace_back( const_cast<char*>( block.payload ) + interval_offset, interval_size );
+      slice.fragments_mem_size += interval_size;
 
       ++offset_index;
     }
@@ -306,8 +314,8 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
       }
 
       auto        sd_index      = to_integral(allen_type);
-      auto&       slice         = std::get<0>( slices[sd_index][slice_index] )[0];
-      auto const& event_offsets = std::get<2>( slices[sd_index][slice_index] );
+      auto&       transposed    = slices[sd_index][slice_index].fragments[0];
+      auto const& event_offsets = slices[sd_index][slice_index].offsets;
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
@@ -319,7 +327,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
 
         // Initialize point to write from offset of previous set
         // All bank offsets are uit32_t so cast to that type
-        auto* banks_write = reinterpret_cast<uint32_t*>( slice.data() + event_offsets[i_event - start_event] );
+        auto* banks_write = reinterpret_cast<uint32_t*>( transposed.data() + event_offsets[i_event - start_event] );
 
         // Where to write the offsets
         auto* banks_offsets_write = banks_write + 1;
-- 
GitLab


From e419eab15584c9254539eae11904a16d1f012c87 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 1 Mar 2022 22:31:41 +0100
Subject: [PATCH 077/120] Store bank sizes

---
 AllenOnline/src/TransposeMEP.cpp | 25 ++++++++++++++++++++++---
 1 file changed, 22 insertions(+), 3 deletions(-)

diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index fb7ff214e..dd6dd67d8 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -220,6 +220,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       auto& fragments = slice.fragments;
       auto& offsets = slice.offsets;
       auto& offsets_size = slice.n_offsets;
+      auto& sizes_offsets = slice.sizes;
 
       // Calculate block offset and size
       size_t interval_offset = 0, interval_size = 0;
@@ -238,9 +239,18 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
         offsets[1]            = event_end - event_start;
         offsets[2 + n_blocks] = 0;
         offset_index          = 0;
+
+        sizes_offsets[offset_index] = 2 * mfp_count[to_integral(allen_type)];
+
         prev_type             = allen_type;
+      } else {
+        sizes_offsets[offset_index] = sizes_offsets[offset_index - 1] + event_end - event_start;
       }
 
+      // Copy bank sizes
+      auto* sizes = reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[offset_index];
+      std::memcpy(sizes, bank_sizes + event_start, event_end - event_start);
+
       // Store source ID
       offsets[2 + offset_index] = source_id;
 
@@ -308,14 +318,23 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
     if ( !bank_types.count( allen_type ) ) {
       prev_type = allen_type;
     } else {
+      auto        sd_index      = to_integral(allen_type);
+      auto&       transposed    = slices[sd_index][slice_index].fragments[0];
+      auto const& event_offsets = slices[sd_index][slice_index].offsets;
+      auto&       sizes_offsets = slices[sd_index][slice_index].sizes;
+
       if ( allen_type != prev_type ) {
         bank_index = 1;
+        auto n_sd_blocks = mfp_count[sd_index];
+        sizes_offsets[bank_index - 1] = 2 * n_sd_blocks;
         prev_type  = allen_type;
+      } else {
+        sizes_offsets[bank_index - 1] = sizes_offsets[bank_index - 2] + end_event - start_event;
       }
 
-      auto        sd_index      = to_integral(allen_type);
-      auto&       transposed    = slices[sd_index][slice_index].fragments[0];
-      auto const& event_offsets = slices[sd_index][slice_index].offsets;
+      // Copy bank sizes
+      auto* sizes = reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[bank_index - 1];
+      std::memcpy(sizes, bank_sizes + start_event, end_event - start_event);
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
-- 
GitLab


From e27381d158585bdc7674f2b0baa006aaa65be75d Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 3 Mar 2022 10:52:44 +0100
Subject: [PATCH 078/120] Fix storage of bank sizes

---
 AllenOnline/src/MEPProvider.cpp  | 1 +
 AllenOnline/src/TransposeMEP.cpp | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 82d011783..d745291e6 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -81,6 +81,7 @@ BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) cons
 
   bno.fragments_mem_size = m_transpose_mep.value() ? slice.offsets[slice.n_offsets - 1] : slice.fragments_mem_size;
   bno.offsets = slice.offsets;
+  bno.sizes = slice.sizes;
   bno.version = m_banks_version[ib];
 
   if (msgLevel(MSG::VERBOSE)) {
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index dd6dd67d8..e888e3902 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -249,7 +249,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
 
       // Copy bank sizes
       auto* sizes = reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[offset_index];
-      std::memcpy(sizes, bank_sizes + event_start, event_end - event_start);
+      std::memcpy(sizes, bank_sizes + event_start, (event_end - event_start) * sizeof(uint16_t));
 
       // Store source ID
       offsets[2 + offset_index] = source_id;
@@ -334,7 +334,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
 
       // Copy bank sizes
       auto* sizes = reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[bank_index - 1];
-      std::memcpy(sizes, bank_sizes + start_event, end_event - start_event);
+      std::memcpy(sizes, bank_sizes + start_event, (end_event - start_event) * sizeof(uint16_t));
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
-- 
GitLab


From cfa223798e2b82bca3d52b4469882d4a2e5da964 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 3 Mar 2022 10:53:44 +0100
Subject: [PATCH 079/120] Extend read_mep for debugging

---
 AllenOnline/application/read_mep.cpp | 128 ++++++++++++++++++++++++---
 1 file changed, 114 insertions(+), 14 deletions(-)

diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 4ce0c7fb6..08f6f9940 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -16,10 +16,13 @@
 #include <unistd.h>
 
 #include <Event/RawBank.h>
+#include <Event/ODIN.h>
 
 #include <Allen/Logger.h>
 #include <Allen/SliceUtils.h>
 #include <Allen/sourceid.h>
+#include <Allen/MEPTools.h>
+#include <Allen/read_mdf.hpp>
 
 #include <MDF/StreamDescriptor.h>
 
@@ -31,6 +34,44 @@
 
 using namespace std;
 
+struct CaloRawBank {
+  uint32_t source_id = 0;
+  uint32_t const* data = nullptr;
+  uint32_t const* end = nullptr;
+
+  // For MEP format
+  CaloRawBank(const uint32_t sid, const char* fragment, const uint16_t s) :
+    source_id {sid}, data {reinterpret_cast<uint32_t const*>(fragment)}, end {reinterpret_cast<uint32_t const*>(fragment + s)}
+  {
+    assert(s % sizeof(uint32_t) == 0);
+  }
+};
+
+struct ODINRawBank {
+
+  uint32_t const* data;
+
+  /// Constructor from MEP layout
+  ODINRawBank(const uint32_t, const char* fragment)
+  {
+    data = reinterpret_cast<uint32_t const*>(fragment);
+  }
+};
+
+std::vector<char> contiguous_mfps(Allen::Slice const& mep_data)
+{
+  // To make direct use of the offsets, the MFPs need to be copied
+  // into temporary storage
+  auto const& mfps = mep_data.fragments;
+  vector<char> mep_fragments(mep_data.fragments_mem_size, '\0');
+  char* destination = &mep_fragments[0];
+  for (gsl::span<char const> mfp : mfps) {
+    ::memcpy(destination, mfp.data(), mfp.size_bytes());
+    destination += mfp.size_bytes();
+  }
+  return mep_fragments;
+}
+
 int main(int argc, char* argv[])
 {
   if (argc != 3) {
@@ -63,7 +104,11 @@ int main(int argc, char* argv[])
 
   Allen::Slices bank_slices;
 
-  std::unordered_set bank_types = {BankTypes::ODIN, BankTypes::Rich};
+  std::unordered_set bank_types = {BankTypes::ODIN, BankTypes::ECal};
+
+  // 160 bytes per fragment, 60 fragments
+  std::vector<char> calo_banks;
+  calo_banks.resize(9600);
 
   for (size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep) {
 
@@ -91,15 +136,54 @@ int main(int argc, char* argv[])
 
     if (i_mep == 0) {
       bank_slices = allocate_slices(1, bank_types,
-        [pf = slice.packing_factor, &mc = mfp_count](auto bt) -> std::tuple<size_t, size_t> {
+        [pf = slice.packing_factor, &mc = mfp_count](auto bt) -> std::tuple<size_t, size_t, size_t> {
           auto n_blocks = mc[to_integral(bt)];
-          return {0ul, 2 + n_blocks + (1 + pf) * (1 + n_blocks) - 2};
+          auto n_sizes = pf * ((Allen::max_fragments + 1) / 2 + 1);
+          return {0ul, n_sizes, 2 + n_blocks + (1 + pf) * (1 + n_blocks) - 2};
         });
     }
     else {
       reset_slice(bank_slices, 0, bank_types, event_ids, true);
     }
 
+    MEP::mep_offsets(bank_slices, 0,
+                     bank_types,
+                     mfp_count,
+                     event_ids,
+                     event_mask,
+                     slice.blocks,
+                     {0ul, slice.packing_factor},
+                     false);
+
+
+    // In MEP layout the fragmets are split into MFPs that are not
+    // contiguous in memory. When the data is copied to the device the
+    // MFPs are copied into device memory back-to-back, making them
+    // contiguous; the offsets are prepared with this in mind.
+
+    auto const& first_block = slice.blocks.front();
+    cout << "MEP with packing: " << std::setw(4) << first_block.header->n_banks
+         << " event_id: " << std::setw(6) << first_block.header->ev_id << "\n";
+
+    // Decode first ODIN
+    auto const odin_index = to_integral(BankTypes::ODIN);
+    auto const& odin_slice = bank_slices[odin_index][0];
+    auto odin_banks = contiguous_mfps(odin_slice);
+    auto const& odin_offsets = odin_slice.offsets;
+    auto const& odin_sizes = odin_slice.sizes;
+    auto odin_bank = MEP::raw_bank<ODINRawBank>(odin_banks.data(), odin_offsets.data(), odin_sizes.data(), 0, 0);
+    LHCb::ODIN odin;
+    if (banks_version[odin_index] == 7) {
+      odin = LHCb::ODIN{{odin_bank.data, 10}};
+    }
+    else {
+      odin = LHCb::ODIN::from_version<6>({odin_bank.data, 10});
+    }
+    cout << "ODIN version: " << banks_version[odin_index]
+         << " run: " << std::setw(7) << odin.runNumber()
+         << " event: " << std::setw(12) << odin.eventNumber() << "\n";
+
+    // Print block information
     for (auto const& block : slice.blocks) {
       // block offsets are in number of 4-byte words
       auto const source_id = block.header->src_id;
@@ -112,26 +196,42 @@ int main(int argc, char* argv[])
         padded_size += bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << block.header->align);
       }
 
-
       cout << "fragment"
            << " magic: 0x" << std::hex << block.header->magic << std::dec
-           << " packing: " << std::setw(4) << block.header->n_banks << " event_id: " << std::setw(6)
-           << block.header->ev_id << " source_id: " << std::setw(6) << source_id
+           << " source_id: " << std::setw(6) << source_id
            << " top5: " << std::setw(2) << SourceId_sys(source_id) << fill << " (" << det << ") "
            << std::setw(5) << SourceId_num(source_id)
            << " version: " << std::setw(2) << unsigned{block.header->block_version}
+           << " align: " << std::setw(3) << pow(2, unsigned{block.header->align})
            << " size: " << std::setw(8) << block.header->bytes() - block.header->header_size()
            << " padded_size: " << std::setw(8) << padded_size << "\n";
     }
 
-    MEP::mep_offsets(bank_slices, 0,
-                     bank_types,
-                     mfp_count,
-                     event_ids,
-                     event_mask,
-                     slice.blocks,
-                     {0ul, slice.packing_factor},
-                     false);
+    auto const ecal_index = to_integral(BankTypes::ECal);
+    auto const& ecal_slice = bank_slices[ecal_index][0];
+    auto ecal_banks = contiguous_mfps(ecal_slice);
+    auto const& ecal_offsets = ecal_slice.offsets;
+    auto const& ecal_sizes = ecal_slice.sizes;
+
+    for (size_t i_event = 0; i_event < 10; ++i_event) {
+      auto* cb = &calo_banks[0];
+      size_t total_size = 0;
+      MEP::RawEvent<CaloRawBank> ecal_event(ecal_banks.data(), ecal_offsets.data(), ecal_sizes.data(), i_event);
+      for (size_t i_bank = 0; i_bank < ecal_event.number_of_raw_banks(); ++i_bank) {
+        auto ecal_bank = ecal_event.raw_bank(i_bank);
+        auto const bank_size = (ecal_bank.end - ecal_bank.data) * sizeof(uint32_t);
+        if (total_size + bank_size > calo_banks.size()) {
+          calo_banks.resize(2 * (total_size + bank_size));
+          cb = &calo_banks[0] + total_size;
+        }
+        total_size += bank_size;
+
+        std::memcpy(cb, ecal_bank.data, bank_size);
+        cb += bank_size;
+      }
+      cout << "\nevent " << std::setw(12) << odin.eventNumber() + i_event << "\n";
+      MDF::dump_hex(calo_banks.data(), total_size);
+    }
   }
   return 0;
 }
-- 
GitLab


From a525301978ea7842ff8b0135b81bbdb281e1d026 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 21 Mar 2022 23:29:47 +0100
Subject: [PATCH 080/120] Fix bank size filling when transposing MEPs

---
 AllenOnline/application/read_mep.cpp | 26 --------------------------
 AllenOnline/src/MEPProvider.cpp      |  2 +-
 AllenOnline/src/TransposeMEP.cpp     | 24 +++++++++++++-----------
 3 files changed, 14 insertions(+), 38 deletions(-)

diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 08f6f9940..b933c7abb 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -206,32 +206,6 @@ int main(int argc, char* argv[])
            << " size: " << std::setw(8) << block.header->bytes() - block.header->header_size()
            << " padded_size: " << std::setw(8) << padded_size << "\n";
     }
-
-    auto const ecal_index = to_integral(BankTypes::ECal);
-    auto const& ecal_slice = bank_slices[ecal_index][0];
-    auto ecal_banks = contiguous_mfps(ecal_slice);
-    auto const& ecal_offsets = ecal_slice.offsets;
-    auto const& ecal_sizes = ecal_slice.sizes;
-
-    for (size_t i_event = 0; i_event < 10; ++i_event) {
-      auto* cb = &calo_banks[0];
-      size_t total_size = 0;
-      MEP::RawEvent<CaloRawBank> ecal_event(ecal_banks.data(), ecal_offsets.data(), ecal_sizes.data(), i_event);
-      for (size_t i_bank = 0; i_bank < ecal_event.number_of_raw_banks(); ++i_bank) {
-        auto ecal_bank = ecal_event.raw_bank(i_bank);
-        auto const bank_size = (ecal_bank.end - ecal_bank.data) * sizeof(uint32_t);
-        if (total_size + bank_size > calo_banks.size()) {
-          calo_banks.resize(2 * (total_size + bank_size));
-          cb = &calo_banks[0] + total_size;
-        }
-        total_size += bank_size;
-
-        std::memcpy(cb, ecal_bank.data, bank_size);
-        cb += bank_size;
-      }
-      cout << "\nevent " << std::setw(12) << odin.eventNumber() + i_event << "\n";
-      MDF::dump_hex(calo_banks.data(), total_size);
-    }
   }
   return 0;
 }
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index d745291e6..f4020335f 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -726,7 +726,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
     auto const sd_index = to_integral(bank_type);
 
     auto aps = eps < 100 ? 100 : eps;
-    auto n_sizes = aps * ((Allen::max_fragments + 1) / 2 + 1);
+    auto n_sizes = aps * ((Allen::max_fragments + 2) / 2 + 1);
 
     if (it == end(BankSizes)) {
       throw std::out_of_range {std::string {"Bank type "} + bank_name(bank_type) + " has no known size"};
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index e888e3902..74f07d1da 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -97,7 +97,6 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
 
     auto const& block = blocks[i_block];
     auto const* bank_sizes     = block.bank_sizes;
-    auto const  align          = block.header->align;
     auto const  allen_type     = source_id_type(block.header->src_id);
     auto const  sd_index       = to_integral(allen_type);
     unsigned    run_number     = 0;
@@ -126,8 +125,8 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
 
         // Anticipate offset structure already here, i.e. don't assign to the first one
         auto  idx           = i - event_start + 1;
-        // Allen raw bank format has the sourceID followed by the raw bank data
-        event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
+        // Allen raw bank format has the sourceID followed by the raw bank data.
+        event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i] + EB::get_padding(bank_sizes[i], sizeof(unsigned));
       }
     }
   }
@@ -308,7 +307,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
         }
         else {
           auto const odin = decode_odin(block.payload, source_offsets[i_event],
-                                        (bank_sizes[i_event] / sizeof(std::uint32_t)), block.header->block_version);
+                                        bank_sizes[i_event], block.header->block_version);
           event_ids.emplace_back( odin.runNumber(), odin.eventNumber() );
           event_mask[i_event - start_event] = 1;
         }
@@ -323,18 +322,21 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
       auto const& event_offsets = slices[sd_index][slice_index].offsets;
       auto&       sizes_offsets = slices[sd_index][slice_index].sizes;
 
+      auto const n_events = end_event - start_event;
+
       if ( allen_type != prev_type ) {
         bank_index = 1;
         auto n_sd_blocks = mfp_count[sd_index];
-        sizes_offsets[bank_index - 1] = 2 * n_sd_blocks;
+
+        for (size_t i = 0; i < n_events; ++i) {
+          sizes_offsets[i] = 2 * (end_event - start_event + 1) + i * n_sd_blocks;
+        }
         prev_type  = allen_type;
-      } else {
-        sizes_offsets[bank_index - 1] = sizes_offsets[bank_index - 2] + end_event - start_event;
       }
 
-      // Copy bank sizes
-      auto* sizes = reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[bank_index - 1];
-      std::memcpy(sizes, bank_sizes + start_event, (end_event - start_event) * sizeof(uint16_t));
+      for (size_t i = 0; i < n_events; ++i) {
+        (reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[i])[bank_index - 1] = bank_sizes[start_event + i];
+      }
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
         // Three things to write for a new set of banks:
@@ -345,7 +347,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
         auto preamble_words = 2 + mfp_count[sd_index];
 
         // Initialize point to write from offset of previous set
-        // All bank offsets are uit32_t so cast to that type
+        // All bank offsets are uint32_t so cast to that type
         auto* banks_write = reinterpret_cast<uint32_t*>( transposed.data() + event_offsets[i_event - start_event] );
 
         // Where to write the offsets
-- 
GitLab


From f55c671e3f6791f93d76d0cdf161eb8d400fede5 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 23 Mar 2022 20:39:01 +0100
Subject: [PATCH 081/120] Fix use of ApplicationMgr in Allen.py

---
 AllenOnline/options/Allen.py | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/Allen.py
index 9ada97805..5ec64ba1e 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/Allen.py
@@ -73,6 +73,8 @@ online_conf.logDeviceFormat = '%TIME%LEVEL%-8NODE: %-32PROCESS %-20SOURCE'
 online_conf.OutputLevel = 3
 online_conf.IOOutputLevel = output_level
 
+appMgr = ApplicationMgr()
+
 if run_online:
     import fifo_log
     online_conf.logDeviceType = 'fifo'
@@ -158,8 +160,9 @@ messageSvc.OutputLevel = output_level
 #     "OnlMonitorSink"
 # ] + appMgr.ExtSvc
 
-# Some extra stuff for timing table
 appMgr.EvtSel = "NONE"
+
+# Key services, order matters!
 appMgr.ExtSvc = [
     'ToolSvc', 'AuditorSvc', allen_conf,
     'Online::Configuration/Application', 'ZeroMQSvc',
-- 
GitLab


From fcb60159fe1ecde712b65247f531c1c2db5f58ff Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 23 Mar 2022 22:30:14 +0100
Subject: [PATCH 082/120] Fill and propagate bank types

---
 AllenOnline/CMakeLists.txt       |  1 -
 AllenOnline/src/MEPProvider.cpp  |  5 +++--
 AllenOnline/src/TransposeMEP.cpp | 22 +++++++++++++++++++---
 3 files changed, 22 insertions(+), 6 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index e6f797b60..4075e24be 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -24,7 +24,6 @@ gaudi_add_library(AllenOnlineLib
         Allen::AllenLib
 )
 
-
 gaudi_add_module(AllenOnlineComp
     SOURCES
         src/AllenApplication.cpp
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index f4020335f..956f2fe78 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -82,6 +82,7 @@ BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) cons
   bno.fragments_mem_size = m_transpose_mep.value() ? slice.offsets[slice.n_offsets - 1] : slice.fragments_mem_size;
   bno.offsets = slice.offsets;
   bno.sizes = slice.sizes;
+  bno.types = slice.types;
   bno.version = m_banks_version[ib];
 
   if (msgLevel(MSG::VERBOSE)) {
@@ -726,14 +727,14 @@ bool MEPProvider::allocate_storage(size_t i_read)
     auto const sd_index = to_integral(bank_type);
 
     auto aps = eps < 100 ? 100 : eps;
-    auto n_sizes = aps * ((Allen::max_fragments + 2) / 2 + 1);
+    auto n_blocks = m_mfp_count[sd_index];
+    auto n_sizes = aps * ((n_blocks + 2) / 2 + 1);
 
     if (it == end(BankSizes)) {
       throw std::out_of_range {std::string {"Bank type "} + bank_name(bank_type) + " has no known size"};
     }
     // In case of direct MEP output, no memory should be allocated.
     if (!m_transpose_mep.value()) {
-      auto n_blocks = m_mfp_count[sd_index];
       // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
       return {0, n_sizes, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
     }
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 74f07d1da..1956930f9 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -7,7 +7,7 @@
 #include <Event/ODIN.h>
 #include <Allen/sourceid.h>
 
-#include <AllenOnline/TransposeMEP.h>
+#include "TransposeMEP.h"
 
 namespace {
   LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes, const uint8_t version ) {
@@ -24,6 +24,11 @@ namespace {
     auto it = Allen::subdetectors.find(static_cast<SourceIdSys>(sd));
     return it == Allen::subdetectors.end() ? BankTypes::Unknown : it->second;
   }
+
+  template<typename T>
+  T* event_entries(gsl::span<unsigned int> offsets, unsigned const event) {
+    return reinterpret_cast<T*>(&offsets[0]) + offsets[i];
+  }
 } // namespace
 
 
@@ -220,6 +225,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       auto& offsets = slice.offsets;
       auto& offsets_size = slice.n_offsets;
       auto& sizes_offsets = slice.sizes;
+      auto& types_offsets = slice.types;
 
       // Calculate block offset and size
       size_t interval_offset = 0, interval_size = 0;
@@ -240,16 +246,22 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
         offset_index          = 0;
 
         sizes_offsets[offset_index] = 2 * mfp_count[to_integral(allen_type)];
+        types_offsets[offset_index] = 4 * mfp_count[to_integral(allen_type)];
 
         prev_type             = allen_type;
       } else {
         sizes_offsets[offset_index] = sizes_offsets[offset_index - 1] + event_end - event_start;
+        types_offsets[offset_index] = types_offsets[offset_index - 1] + event_end - event_start;
       }
 
       // Copy bank sizes
-      auto* sizes = reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[offset_index];
+      auto* sizes = event_entries<uint16_t>(sizes_offsets);
       std::memcpy(sizes, bank_sizes + event_start, (event_end - event_start) * sizeof(uint16_t));
 
+      // Copy bank types
+      auto* types = event_entries<uint8_t>(types_offsets);
+      std::memcpy(types, bank_types + event_start, (event_end - event_start));
+
       // Store source ID
       offsets[2 + offset_index] = source_id;
 
@@ -296,6 +308,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
     auto        allen_type     = source_id_type(source_id);
     auto&       source_offsets = input_offsets[i_block];
     auto const* bank_sizes     = block.bank_sizes;
+    auto const* bank_types     = block.bank_types;
 
     // Check what to do with this bank
     if ( allen_type == BankTypes::ODIN ) {
@@ -321,6 +334,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
       auto&       transposed    = slices[sd_index][slice_index].fragments[0];
       auto const& event_offsets = slices[sd_index][slice_index].offsets;
       auto&       sizes_offsets = slices[sd_index][slice_index].sizes;
+      auto&       types_offsets = slices[sd_index][slice_index].types;
 
       auto const n_events = end_event - start_event;
 
@@ -330,12 +344,14 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
 
         for (size_t i = 0; i < n_events; ++i) {
           sizes_offsets[i] = 2 * (end_event - start_event + 1) + i * n_sd_blocks;
+          types_offsets[i] = 4 * (end_event - start_event + 1) + i * n_sd_blocks;
         }
         prev_type  = allen_type;
       }
 
       for (size_t i = 0; i < n_events; ++i) {
-        (reinterpret_cast<uint16_t*>(&sizes_offsets[0]) + sizes_offsets[i])[bank_index - 1] = bank_sizes[start_event + i];
+        event_entries<uint16_t>(sizes_offsets)[bank_index - 1] = bank_sizes[start_event + i];
+        event_entries<uint8_t>(types_offsets)[bank_index - 1] = bank_types[start_event + i];
       }
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
-- 
GitLab


From 6077e68843a1aedfd5c6a22a6d2c2dbc6a708d7e Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 29 Mar 2022 11:00:39 +0200
Subject: [PATCH 083/120] Fix build

---
 .../application/bench_mep_provider.cpp        |  66 ----
 AllenOnline/application/mdf_to_mep.cpp        | 287 ------------------
 AllenOnline/application/read_mep.cpp          |  11 +-
 AllenOnline/src/TransposeMEP.cpp              |  25 +-
 4 files changed, 20 insertions(+), 369 deletions(-)
 delete mode 100644 AllenOnline/application/bench_mep_provider.cpp
 delete mode 100644 AllenOnline/application/mdf_to_mep.cpp

diff --git a/AllenOnline/application/bench_mep_provider.cpp b/AllenOnline/application/bench_mep_provider.cpp
deleted file mode 100644
index 1bf0d7fbc..000000000
--- a/AllenOnline/application/bench_mep_provider.cpp
+++ /dev/null
@@ -1,66 +0,0 @@
-/*****************************************************************************\
-* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
-\*****************************************************************************/
-#include <cstring>
-#include <iostream>
-#include <fstream>
-#include <string>
-#include <iomanip>
-#include <unordered_set>
-#include <map>
-
-#include <Event/RawBank.h>
-#include <Timer.h>
-#include <MEPProvider.h>
-
-using namespace std;
-
-int main(int argc, char* argv[])
-{
-  if (argc <= 1) {
-    cout << "usage: bench_provider <file.mep> <file.mep> <file.mep> ..." << endl;
-    return -1;
-  }
-
-  string filename = {argv[1]};
-  size_t n_slices = 10;
-  size_t events_per_slice = 1000;
-  double n_filled = 0.;
-
-  vector<string> files(argc - 1);
-  for (int i = 0; i < argc - 1; ++i) {
-    files[i] = argv[i + 1];
-  }
-
-  logger::setVerbosity(4);
-
-  Timer t;
-
-  MEPProviderConfig config {false,         // verify MEP checksums
-                            10,            // number of read buffers
-                            2,             // number of transpose threads
-                            4,             // MPI sliding window size
-                            false,         // Receive from MPI or read files
-                            false,         // Run the application non-stop
-                            true,          // Transpose MEP
-                            false,         // Split by run number
-                            {{"mem", 0}}}; // mapping of receiver to its numa node
-
-  MEPProvider<BankTypes::VP, BankTypes::UT, BankTypes::FT, BankTypes::MUON> mep {
-    n_slices, events_per_slice, {}, files, config};
-
-  chrono::milliseconds sleep_interval {10};
-
-  bool good = true, done = false, timed_out = false;
-  size_t filled = 0, slice = 0;
-  uint runno = 0;
-  while (good || filled != 0) {
-    std::tie(good, done, timed_out, slice, filled, runno) = mep.get_slice();
-    n_filled += filled;
-    this_thread::sleep_for(sleep_interval);
-    mep.slice_free(slice);
-  }
-
-  t.stop();
-  cout << "Filled " << n_filled / t.get() << " events/s\n";
-}
diff --git a/AllenOnline/application/mdf_to_mep.cpp b/AllenOnline/application/mdf_to_mep.cpp
deleted file mode 100644
index f42094fb4..000000000
--- a/AllenOnline/application/mdf_to_mep.cpp
+++ /dev/null
@@ -1,287 +0,0 @@
-/*****************************************************************************\
-* (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
-\*****************************************************************************/
-#include <cstring>
-#include <iostream>
-#include <fstream>
-#include <string>
-#include <iomanip>
-#include <unordered_set>
-#include <numeric>
-#include <map>
-#include <cassert>
-
-#include <unistd.h>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <fcntl.h>
-
-#include <Event/RawBank.h>
-#include <read_mdf.hpp>
-#include <eb_header.hpp>
-#include <Transpose.h>
-#include <Common.h>
-#include <BankTypes.h>
-
-using namespace std;
-
-namespace detail {
-
-  template<typename T>
-  std::ostream& write(std::ostream& os, const T& t)
-  {
-    // if you would like to know why there is a check for trivially copyable,
-    // please read the 'notes' section of https://en.cppreference.com/w/cpp/types/is_trivially_copyable
-    if constexpr (gsl::details::is_span<T>::value) {
-      return os.write(reinterpret_cast<const char*>(t.data()), t.size_bytes());
-    }
-    else if constexpr (std::is_trivially_copyable_v<T> && !gsl::details::is_span<T>::value) {
-      return os.write(reinterpret_cast<const char*>(&t), sizeof(T));
-    }
-    else {
-      static_assert(std::is_trivially_copyable_v<typename T::value_type>);
-      return write(os, as_bytes(gsl::make_span(t)));
-    }
-  }
-
-} // namespace detail
-
-class FileWriter {
-  std::ofstream m_f;
-
-public:
-  FileWriter(const std::string& name) : m_f {name, std::ios::out | std::ios::binary} {}
-
-  bool is_open() { return m_f.is_open(); }
-
-  template<typename... Args>
-  FileWriter& write(Args&&... args)
-  {
-    (detail::write(m_f, std::forward<Args>(args)), ...);
-    return *this;
-  }
-};
-
-int main(int argc, char* argv[])
-{
-
-  if (argc < 4) {
-    cout << "usage: mdf_to_mep output_file #MEPs packing_factor input.mdf ...\n";
-    return -1;
-  }
-
-  string output_file {argv[1]};
-  size_t n_meps = std::atol(argv[2]);
-  uint16_t packing_factor = std::atoi(argv[3]);
-
-  vector<string> input_files(argc - 4);
-  for (int i = 0; i < argc - 4; ++i) {
-    input_files[i] = argv[i + 4];
-  }
-
-  vector<char> buffer(1024 * 1024, '\0');
-  vector<char> decompression_buffer(1024 * 1024, '\0');
-
-  LHCb::MDFHeader mdf_header;
-  bool error = false;
-  bool eof = false;
-  gsl::span<char> bank_span;
-
-  bool sizes_known = false;
-  bool count_success = false;
-  std::array<unsigned int, LHCb::NBankTypes> banks_count;
-
-  size_t n_read = 0;
-  size_t n_written = 0;
-  uint64_t event_id = 0;
-
-  std::vector<std::tuple<EB::BlockHeader, size_t, vector<char>>> blocks;
-  EB::Header mep_header;
-
-  // offsets to fragments of the detector types
-  std::array<size_t, LHCb::NBankTypes> block_offsets {0};
-
-  // Header version 3
-  auto hdr_size = LHCb::MDFHeader::sizeOf(3);
-  std::vector<char> header_buffer(hdr_size, '\0');
-  auto* header = reinterpret_cast<LHCb::MDFHeader*>(header_buffer.data());
-  header->setHeaderVersion(3);
-  header->setDataType(LHCb::MDFHeader::BODY_TYPE_MEP);
-  header->setSubheaderLength(hdr_size - sizeof(LHCb::MDFHeader));
-
-  FileWriter writer {output_file};
-  if (!writer.is_open()) {
-    cerr << "Failed to open output file: " << strerror(errno) << "\n";
-    return -1;
-  }
-
-  auto write_fragments = [&writer, &blocks, &n_written, &mep_header, hdr_size, packing_factor, header] {
-    header->setSize(
-      mep_header.header_size(blocks.size()) +
-      std::accumulate(blocks.begin(), blocks.end(), 0, [packing_factor](size_t s, const auto& entry) {
-        auto& [block_header, n_filled, data] = entry;
-        return s + block_header.header_size(packing_factor) + block_header.block_size;
-      }));
-    writer.write(gsl::span {reinterpret_cast<char const*>(header), hdr_size});
-
-    size_t block_offset = 0;
-    for (size_t ib = 0; ib < blocks.size(); ++ib) {
-      mep_header.offsets[ib] = block_offset;
-      auto const& block = std::get<0>(blocks[ib]);
-      block_offset += block.header_size(block.n_frag) + block.block_size;
-    }
-    mep_header.mep_size = block_offset;
-    mep_header.packing_factor = packing_factor;
-
-    writer.write(
-      mep_header.n_blocks,
-      mep_header.packing_factor,
-      mep_header.reserved,
-      mep_header.mep_size,
-      mep_header.source_ids,
-      mep_header.versions,
-      mep_header.offsets);
-
-    for (auto& [block_header, n_filled, data] : blocks) {
-      assert(std::accumulate(block_header.sizes.begin(), block_header.sizes.end(), 0u) == block_header.block_size);
-      writer.write(
-        block_header.event_id,
-        block_header.n_frag,
-        block_header.reserved,
-        block_header.block_size,
-        block_header.types,
-        block_header.sizes);
-      writer.write(gsl::span {data.data(), block_header.block_size});
-
-      // Reset the fragments
-      block_header.block_size = 0;
-      n_filled = 0;
-    }
-    ++n_written;
-  };
-
-  for (auto const& file : input_files) {
-    auto input = MDF::open(file.c_str(), O_RDONLY);
-    if (input.good) {
-      cout << "Opened " << file << "\n";
-    }
-    else {
-      cerr << "Failed to open " << file << " " << strerror(errno) << "\n";
-      error = true;
-      break;
-    }
-    while (!eof && n_written < n_meps) {
-      std::tie(eof, error, bank_span) = MDF::read_event(input, mdf_header, buffer, decompression_buffer, false);
-      if (eof) {
-        eof = false;
-        break;
-      }
-      else if (error) {
-        cerr << "Failed to read event\n";
-        return -1;
-      }
-      else {
-        ++n_read;
-      }
-
-      if (!sizes_known) {
-        // Count the number of banks of each type and the start of the
-        // source ID range
-        std::tie(count_success, banks_count) = fill_counts(bank_span);
-        // Skip DAQ bank
-        uint16_t n_blocks =
-          std::accumulate(banks_count.begin(), banks_count.end(), 0) - banks_count[LHCb::RawBank::DAQ];
-        size_t offset = 0, i = 0;
-        for (i = 0; i < banks_count.size(); ++i) {
-          if (i != to_integral(LHCb::RawBank::DAQ)) {
-            block_offsets[i] = offset;
-            offset += banks_count[i];
-          }
-        }
-        blocks.resize(n_blocks);
-        for (auto& block : blocks) {
-          std::get<2>(block).resize(packing_factor * average_event_size * kB / n_blocks);
-        }
-
-        mep_header = EB::Header {packing_factor, n_blocks};
-        sizes_known = true;
-      }
-
-      // Put the banks in the event-local buffers
-      char const* bank = bank_span.data();
-      char const* end = bank + bank_span.size();
-      size_t source_offset = 0;
-      auto prev_type = LHCb::RawBank::L0Calo;
-      while (bank < end) {
-        const auto* b = reinterpret_cast<const LHCb::RawBank*>(bank);
-        if (b->magic() != LHCb::RawBank::MagicPattern) {
-          cout << "magic pattern failed: " << std::hex << b->magic() << std::dec << endl;
-          return -1;
-        }
-
-        // Skip the DAQ bank, it's created on read from the MDF header
-        if (b->type() < LHCb::RawBank::LastType && b->type() != LHCb::RawBank::DAQ) {
-          if (b->type() != prev_type) {
-            source_offset = 0;
-            prev_type = b->type();
-          }
-          else {
-            ++source_offset;
-          }
-          auto block_index = block_offsets[b->type()] + source_offset;
-          auto& [block_header, n_filled, data] = blocks[block_index];
-
-          if (n_filled == 0) {
-            mep_header.source_ids[block_index] = b->sourceID();
-            mep_header.versions[block_index] = b->version();
-            block_header = EB::BlockHeader {event_id, packing_factor};
-          }
-          else if (mep_header.source_ids[block_index] != b->sourceID()) {
-            cout << "Error: banks not ordered in the same way: " << mep_header.source_ids[block_index] << " "
-                 << b->sourceID() << "\n";
-            return -1;
-          }
-
-          // NOTE: All banks are truncated to 32 bit values. This
-          // doesn't seem to make a difference except for the UT,
-          // where the size is larger than the number of words o.O
-          auto n_words = b->size() / sizeof(uint32_t);
-          auto word_size = n_words * sizeof(uint32_t);
-          block_header.types[n_filled] = b->type();
-          block_header.sizes[n_filled] = n_words * sizeof(uint32_t);
-
-          // Resize on demand
-          if (block_header.block_size + word_size >= data.size()) {
-            data.resize(std::max(static_cast<size_t>(1.5 * data.size()), data.size() + word_size));
-          }
-
-          // Copy bank data
-          ::memcpy(&data[0] + block_header.block_size, b->data(), word_size);
-          block_header.block_size += word_size;
-
-          ++n_filled;
-        }
-        else if (b->type() != LHCb::RawBank::DAQ) {
-          cout << "unknown bank type: " << b->type() << endl;
-        }
-
-        // Move to next raw bank
-        bank += b->totalSize();
-      }
-
-      if (n_read % packing_factor == 0 && n_read != 0) {
-        write_fragments();
-        event_id += packing_factor;
-      }
-    }
-
-    input.close();
-    if (n_written >= n_meps) break;
-  }
-
-  if (!error) {
-    cout << "Wrote " << n_written << " MEPs with " << (n_read % packing_factor) << " events left over.\n";
-  }
-
-  return error ? -1 : 0;
-}
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index b933c7abb..25235902a 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -49,12 +49,14 @@ struct CaloRawBank {
 
 struct ODINRawBank {
 
-  uint32_t const* data;
+  uint32_t const* data = nullptr;
+  uint16_t size = 0;
 
   /// Constructor from MEP layout
-  ODINRawBank(const uint32_t, const char* fragment)
+  ODINRawBank(const uint32_t, const char* fragment, uint16_t s)
   {
     data = reinterpret_cast<uint32_t const*>(fragment);
+    size = s;
   }
 };
 
@@ -138,7 +140,7 @@ int main(int argc, char* argv[])
       bank_slices = allocate_slices(1, bank_types,
         [pf = slice.packing_factor, &mc = mfp_count](auto bt) -> std::tuple<size_t, size_t, size_t> {
           auto n_blocks = mc[to_integral(bt)];
-          auto n_sizes = pf * ((Allen::max_fragments + 1) / 2 + 1);
+          auto n_sizes = pf * ((n_blocks + 1) / 2 + 1);
           return {0ul, n_sizes, 2 + n_blocks + (1 + pf) * (1 + n_blocks) - 2};
         });
     }
@@ -171,7 +173,8 @@ int main(int argc, char* argv[])
     auto odin_banks = contiguous_mfps(odin_slice);
     auto const& odin_offsets = odin_slice.offsets;
     auto const& odin_sizes = odin_slice.sizes;
-    auto odin_bank = MEP::raw_bank<ODINRawBank>(odin_banks.data(), odin_offsets.data(), odin_sizes.data(), 0, 0);
+    auto const& odin_types = odin_slice.types;
+    auto odin_bank = MEP::raw_bank<ODINRawBank>(odin_banks.data(), odin_offsets.data(), odin_sizes.data(), odin_types.data(), 0, 0);
     LHCb::ODIN odin;
     if (banks_version[odin_index] == 7) {
       odin = LHCb::ODIN{{odin_bank.data, 10}};
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 1956930f9..852a818b6 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -7,7 +7,7 @@
 #include <Event/ODIN.h>
 #include <Allen/sourceid.h>
 
-#include "TransposeMEP.h"
+#include <AllenOnline/TransposeMEP.h>
 
 namespace {
   LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes, const uint8_t version ) {
@@ -27,7 +27,7 @@ namespace {
 
   template<typename T>
   T* event_entries(gsl::span<unsigned int> offsets, unsigned const event) {
-    return reinterpret_cast<T*>(&offsets[0]) + offsets[i];
+    return reinterpret_cast<T*>(&offsets[0]) + offsets[event];
   }
 } // namespace
 
@@ -167,7 +167,7 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
 }
 
 std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int const slice_index,
-                                                 std::unordered_set<BankTypes> const& bank_types,
+                                                 std::unordered_set<BankTypes> const& subdetectors,
                                                  std::array<unsigned int, NBankTypes> const& mfp_count,
                                                  EventIDs& event_ids, std::vector<char>& event_mask,
                                                  MEP::Blocks const& blocks,
@@ -186,6 +186,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
     auto const  sd_index   = to_integral(allen_type);
     auto        n_blocks   = mfp_count[sd_index];
     auto const* bank_sizes = block.bank_sizes;
+    auto const* bank_types = block.bank_types;
     auto const  align      = block.header->align;
 
     // Decode ODIN banks
@@ -194,7 +195,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       unsigned fragment_offset = 0;
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
         if ( i_event >= event_start) {
-          if (block.bank_types[i_event] == LHCb::RawBank::ODIN) {
+          if (bank_types[i_event] == LHCb::RawBank::ODIN) {
             auto const odin = decode_odin(block.payload, fragment_offset,
                                           bank_sizes[i_event], block.header->block_version);
             unsigned const odin_run_number = odin.runNumber();
@@ -219,7 +220,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       }
     }
 
-    if ( bank_types.count( allen_type ) ) {
+    if ( subdetectors.count( allen_type ) ) {
       auto& slice = slices[sd_index][slice_index];
       auto& fragments = slice.fragments;
       auto& offsets = slice.offsets;
@@ -255,11 +256,11 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       }
 
       // Copy bank sizes
-      auto* sizes = event_entries<uint16_t>(sizes_offsets);
+      auto* sizes = event_entries<uint16_t>(sizes_offsets, offset_index);
       std::memcpy(sizes, bank_sizes + event_start, (event_end - event_start) * sizeof(uint16_t));
 
       // Copy bank types
-      auto* types = event_entries<uint8_t>(types_offsets);
+      auto* types = event_entries<uint8_t>(types_offsets, offset_index);
       std::memcpy(types, bank_types + event_start, (event_end - event_start));
 
       // Store source ID
@@ -291,7 +292,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
 }
 
 bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
-                         std::unordered_set<BankTypes> const& bank_types,
+                         std::unordered_set<BankTypes> const& subdetectors,
                          std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
                          std::vector<char>& event_mask,
                          EB::MEP const* mep, MEP::Blocks const& blocks,
@@ -314,7 +315,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
     if ( allen_type == BankTypes::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
       for ( uint16_t i_event = start_event; i_event < end_event; ++i_event ) {
-        if (block.bank_types[i_event] != LHCb::RawBank::ODIN) {
+        if (bank_types[i_event] != LHCb::RawBank::ODIN) {
           event_ids.emplace_back( 0, 0 );
           event_mask[i_event - start_event] = 0;
         }
@@ -327,7 +328,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
       }
     }
 
-    if ( !bank_types.count( allen_type ) ) {
+    if ( !subdetectors.count( allen_type ) ) {
       prev_type = allen_type;
     } else {
       auto        sd_index      = to_integral(allen_type);
@@ -350,8 +351,8 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
       }
 
       for (size_t i = 0; i < n_events; ++i) {
-        event_entries<uint16_t>(sizes_offsets)[bank_index - 1] = bank_sizes[start_event + i];
-        event_entries<uint8_t>(types_offsets)[bank_index - 1] = bank_types[start_event + i];
+        event_entries<uint16_t>(sizes_offsets, i)[bank_index - 1] = bank_sizes[start_event + i];
+        event_entries<uint8_t>(types_offsets, i)[bank_index - 1] = bank_types[start_event + i];
       }
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
-- 
GitLab


From 351cf46574e48fe471b57948d435484e283af3ef Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 13 Apr 2022 14:55:32 +0200
Subject: [PATCH 084/120] Improve handling of MEP files with errors

---
 AllenOnline/src/MEPProvider.cpp | 18 +++++++++---------
 AllenOnline/src/ReadMEP.cpp     |  2 +-
 2 files changed, 10 insertions(+), 10 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 956f2fe78..e65a5153d 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -951,13 +951,7 @@ void MEPProvider::mep_read()
         }
       }
 
-      if (!success) {
-        // Error encountered
-        read_error();
-        break;
-      }
-
-      if (!eof) {
+      if (!eof && success) {
         if (to_read) {
           to_publish = std::min(*to_read, size_t {slice.packing_factor});
           *to_read -= to_publish;
@@ -967,7 +961,7 @@ void MEPProvider::mep_read()
         }
       }
 
-      if ((to_read && *to_read == 0) || (eof && !open_file())) {
+      if ((success && to_read && *to_read == 0) || (eof && !open_file())) {
         // Try to open the next file, if there is none, prefetching
         // is done.
         if (!m_read_error && msgLevel(MSG::DEBUG)) {
@@ -980,6 +974,12 @@ void MEPProvider::mep_read()
         }
         break;
       }
+
+      if (!eof && !success) {
+        // Error encountered
+        read_error();
+        break;
+      }
     }
 
     if (m_read_error || (!m_sizes_known && !allocate_storage(i_buffer))) {
@@ -990,7 +990,7 @@ void MEPProvider::mep_read()
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
-    if (success) {
+    if (!m_read_error) {
       {
         std::unique_lock<std::mutex> lock {m_buffer_mutex};
 
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index 38739ced1..015b0b439 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -46,7 +46,7 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   int read_sc = input.read(&buffer[0], sizeof(EB::MEP_header));
   if (read_sc == 0) {
     cout << "Cannot read more data (Header). End-of-File reached.\n";
-    return {true, true, nullptr, 0u, {}};
+    return {true, false, nullptr, 0u, {}};
   }
   else if (read_sc < 0) {
     cerr << "Failed to read header " << strerror(errno) << "\n";
-- 
GitLab


From 2e23ec73863f333cb9879eff26821f43e82221a8 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 13 Apr 2022 14:56:09 +0200
Subject: [PATCH 085/120] Fix AllenApplication for direct file output

---
 AllenOnline/src/AllenApplication.cpp | 32 ++++++++++++++--------------
 AllenOnline/src/AllenApplication.h   |  3 +--
 2 files changed, 17 insertions(+), 18 deletions(-)

diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 4a8f9ba29..853917c7e 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -200,7 +200,17 @@ int AllenApplication::configureApplication()
   // Use a deleter that does nothing to allow wrapping it in a shared pointer.
   m_provider.reset(provider, [](IInputProvider*) {});
 
-  m_json = m_allenConfig->json;
+  auto json = m_allenConfig->json;
+  auto paramDir = resolveEnvVars(m_allenConfig->paramDir);
+
+  m_options = {{"v", std::to_string(6 - m_config->outputLevel())},
+               {"t", m_allenConfig->nThreads.toString()},
+               {"params", paramDir},
+               {"device", m_allenConfig->device.value()},
+               {"s", std::to_string(m_nSlices)},
+               {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())},
+               {"sequence", json},
+               {"run-from-json", "1"}};
 
   m_output = makeOutput();
   if (!m_allenConfig->output.value().empty() && !m_output) {
@@ -292,19 +302,9 @@ int AllenApplication::continueProcessing()
 
 void AllenApplication::allenLoop()
 {
-  auto paramDir = resolveEnvVars(m_allenConfig->paramDir);
-
   //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
   //--device 23:00.0
-  std::map<std::string, std::string> allen_options = {{"v", std::to_string(6 - m_config->outputLevel())},
-                                                      {"t", m_allenConfig->nThreads.toString()},
-                                                      {"params", paramDir},
-                                                      {"device", m_allenConfig->device.value()},
-                                                      {"s", std::to_string(m_nSlices)},
-                                                      {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())},
-                                                      {"sequence", m_allenConfig->json.value()},
-                                                      {"run-from-json", "1"}};
-  allen(allen_options, this, m_provider, m_output, m_zmqSvc.get(), m_controlConnection);
+  allen(m_options, this, m_provider, m_output, m_zmqSvc.get(), m_controlConnection);
 }
 
 void AllenApplication::update(unsigned long run)
@@ -344,7 +344,7 @@ OutputHandler* AllenApplication::makeOutput()
   }
 
   SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
-  auto outputSvc = sloc->service("Allen::MBMOutput/MBMOutput");
+  auto outputSvc = sloc->service("Allen::MBMOutput/MBMOutput", false);
 
   auto output = m_allenConfig->output.value();
   if (!outputSvc && output.empty()) {
@@ -362,12 +362,12 @@ OutputHandler* AllenApplication::makeOutput()
   }
   else if (!output.empty()) {
     output_type = output.substr(0, p);
-    connection = output.substr(p + 3, std::string::npos);
+    connection = output;
   }
 
   if (output_type == "file" || output_type == "tcp") {
-    std::map<std::string, std::string> options = {{"configuration", m_json},
-                                                  {"output-file", output}};
+    auto options = m_options;
+    options["output-file"] = connection;
     m_outputHolder = Allen::output_handler(m_provider.get(), m_zmqSvc.get(), std::move(options));
     return m_outputHolder.get();
   }
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 2eeac8c02..6f6baf170 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -97,6 +97,7 @@ private:
 
   std::string const m_controlConnection = "inproc://AllenApplicationControl";
   std::string const m_runChangeConnection = "inproc://AllenApplicationRunChange";
+  std::map<std::string, std::string> m_options;
 
   std::shared_ptr<IInputProvider> m_provider;
   std::unique_ptr<OutputHandler> m_outputHolder;
@@ -112,6 +113,4 @@ private:
 
   std::thread m_allenThread;
   std::optional<zmq::socket_t> m_allenControl = std::nullopt;
-
-  std::string m_json;
 };
-- 
GitLab


From 7d561a746846b7538e9fce3107c8fb45cf1acc2a Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 21 Apr 2022 10:39:04 +0200
Subject: [PATCH 086/120] Fix status.work_counter accounting.

---
 AllenOnline/src/MEPProvider.cpp | 50 ++++++++++++++++-----------------
 AllenOnline/src/MEPProvider.h   |  2 +-
 2 files changed, 26 insertions(+), 26 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index e65a5153d..119da7126 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -160,6 +160,8 @@ void MEPProvider::slice_free(size_t slice_index)
       // Clear relation between slice and buffer
       i_buffer = std::get<0>(m_slice_to_buffer[slice_index]);
       auto& status = m_buffer_status[i_buffer];
+      --status.work_counter;
+
       m_slice_to_buffer[slice_index] = {-1, 0, 0};
 
       // If MEPs are not transposed and the respective buffer is no
@@ -178,6 +180,17 @@ void MEPProvider::slice_free(size_t slice_index)
             m_buffer_event[i_buffer] = false;
           }
         }
+
+        if (status.work_counter == 0) {
+          m_transpose_done =
+            m_done && std::all_of(m_buffer_status.begin(), m_buffer_status.end(), [](BufferStatus const& stat) {
+              return stat.intervals.empty() && stat.work_counter == 0;
+            });
+          if (m_transpose_done) {
+            m_transpose_cond.notify_all();
+          }
+        }
+
         if (msgLevel(MSG::DEBUG)) {
           debug() << "Freed MEP buffer " << i_buffer << "; writable: " << count_writable() << endmsg;
           for (auto const& status :  m_buffer_status) {
@@ -828,9 +841,12 @@ std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvi
   return {it, distance(m_buffer_status.begin(), it)};
 }
 
-void MEPProvider::set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events)
+void MEPProvider::set_intervals(BufferStatus& status, size_t n_events)
 {
   if (n_events == 0) return;
+
+  auto& intervals = status.intervals;
+
   size_t const eps = m_events_per_slice.value();
   auto n_interval = n_events / eps;
   auto rest = n_events % eps;
@@ -846,6 +862,7 @@ void MEPProvider::set_intervals(std::vector<std::tuple<size_t, size_t>>& interva
     }
     intervals.emplace_back((i - 1) * eps, i * eps);
   }
+  status.work_counter = intervals.size();
 }
 
 // mep reader thread
@@ -1006,7 +1023,7 @@ void MEPProvider::mep_read()
           (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
 
           // Set intervals for offset calculation/transposition
-          set_intervals(status.intervals, n_events);
+          set_intervals(status, n_events);
         }
         else {
           // We didn't read anything, so free the buffer we got again
@@ -1247,7 +1264,7 @@ void MEPProvider::mpi_read()
     if (!mpi_error) {
       {
         std::unique_lock<std::mutex> lock {m_buffer_mutex};
-        set_intervals(m_buffer_status[i_buffer].intervals, size_t {slice.packing_factor});
+        set_intervals(m_buffer_status[i_buffer], size_t {slice.packing_factor});
         assert(m_buffer_status[i_buffer].work_counter == 0);
       }
       if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
@@ -1401,7 +1418,7 @@ void MEPProvider::bm_read(const std::string& buffer_name)
       auto& status = m_buffer_status[i_buffer];
       assert(status.work_counter == 0);
 
-      set_intervals(status.intervals, size_t {slice.packing_factor});
+      set_intervals(status, size_t {slice.packing_factor});
     }
 
     if (!cancelled) {
@@ -1457,9 +1474,6 @@ void MEPProvider::transpose(int thread_id)
       interval = status.intervals.back();
       status.intervals.pop_back();
 
-      ++(status.work_counter);
-      status.writable = false;
-
       if (msgLevel(MSG::DEBUG)) {
         debug() << "Transpose " << thread_id << ": Got MEP slice index " << i_buffer << " interval [" << std::get<0>(interval)
                 << "," << std::get<1>(interval) << ")" << endmsg;
@@ -1489,7 +1503,9 @@ void MEPProvider::transpose(int thread_id)
         *it = false;
         slice_index = distance(m_slice_free.begin(), it);
         if (msgLevel(MSG::DEBUG)) debug() << "Transpose " << thread_id << ": Got slice index " << *slice_index << endmsg;
-
+      }
+      {
+        std::unique_lock<std::mutex> lock {m_buffer_mutex};
         // Keep track of what buffer this slice belonged to
         m_slice_to_buffer[*slice_index] = {i_buffer, std::get<0>(interval), std::get<1>(interval)};
       }
@@ -1544,8 +1560,6 @@ void MEPProvider::transpose(int thread_id)
 
     if (m_read_error || !good) {
       std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      auto& status = m_buffer_status[i_buffer];
-      --status.work_counter;
       m_read_error = true;
       m_transpose_cond.notify_one();
       break;
@@ -1559,24 +1573,10 @@ void MEPProvider::transpose(int thread_id)
     m_transposed_cond.notify_one();
     slice_index.reset();
 
-    // Check if the read buffer is now empty. If it is, it can be
-    // reused, otherwise give it to another transpose thread once a
-    // new target slice is available
-    {
+    if (n_transposed != std::get<1>(interval) - std::get<0>(interval)) {
       std::unique_lock<std::mutex> lock {m_buffer_mutex};
-
       auto& status = m_buffer_status[i_buffer];
-      --status.work_counter;
-
-      if (n_transposed != std::get<1>(interval) - std::get<0>(interval)) {
         status.intervals.emplace_back(std::get<0>(interval) + n_transposed, std::get<1>(interval));
-      }
-      else if (status.work_counter == 0) {
-        m_transpose_done =
-          m_done && std::all_of(m_buffer_status.begin(), m_buffer_status.end(), [](BufferStatus const& stat) {
-            return stat.intervals.empty() && stat.work_counter == 0;
-          });
-      }
     }
   }
 }
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index 01ea205d7..fed2cdd8e 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -174,7 +174,7 @@ private:
     std::condition_variable& cond,
     std::unique_lock<std::mutex>& lock);
 
-  void set_intervals(std::vector<std::tuple<size_t, size_t>>& intervals, size_t n_events);
+  void set_intervals(BufferStatus& intervals, size_t n_events);
 
   // mep reader thread
   void mep_read();
-- 
GitLab


From b989d8918060633d416c0690b012f552789c1610 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 21 Apr 2022 11:02:13 +0200
Subject: [PATCH 087/120] Don't do input MEP related things in the transpose
 threads.

---
 AllenOnline/src/MEPProvider.cpp | 30 +++++++++++++++---------------
 AllenOnline/src/MEPProvider.h   |  2 +-
 2 files changed, 16 insertions(+), 16 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 119da7126..a19b885c7 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -841,10 +841,21 @@ std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvi
   return {it, distance(m_buffer_status.begin(), it)};
 }
 
-void MEPProvider::set_intervals(BufferStatus& status, size_t n_events)
+void MEPProvider::prepare_mep(size_t i_buffer, size_t n_events)
 {
+  auto& slice = m_net_slices[i_buffer];
+
+  // Fill blocks
+  MEP::find_blocks(slice.mep, slice.blocks);
+
+  // Fill fragment offsets
+  MEP::fragment_offsets(slice.blocks, slice.offsets);
+
   if (n_events == 0) return;
 
+  auto& status = m_buffer_status[i_buffer];
+  assert(status.work_counter == 0);
+
   auto& intervals = status.intervals;
 
   size_t const eps = m_events_per_slice.value();
@@ -1023,7 +1034,7 @@ void MEPProvider::mep_read()
           (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
 
           // Set intervals for offset calculation/transposition
-          set_intervals(status, n_events);
+          prepare_mep(i_buffer, n_events);
         }
         else {
           // We didn't read anything, so free the buffer we got again
@@ -1264,8 +1275,7 @@ void MEPProvider::mpi_read()
     if (!mpi_error) {
       {
         std::unique_lock<std::mutex> lock {m_buffer_mutex};
-        set_intervals(m_buffer_status[i_buffer], size_t {slice.packing_factor});
-        assert(m_buffer_status[i_buffer].work_counter == 0);
+        prepare_mep(i_buffer, size_t {slice.packing_factor});
       }
       if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
       m_transpose_cond.notify_one();
@@ -1414,11 +1424,7 @@ void MEPProvider::bm_read(const std::string& buffer_name)
     // ready. If prefetching is done, wake up all threads
     if (!cancelled) {
       std::unique_lock<std::mutex> lock {m_buffer_mutex};
-
-      auto& status = m_buffer_status[i_buffer];
-      assert(status.work_counter == 0);
-
-      set_intervals(status, size_t {slice.packing_factor});
+      prepare_mep(i_buffer, size_t {slice.packing_factor});
     }
 
     if (!cancelled) {
@@ -1519,12 +1525,6 @@ void MEPProvider::transpose(int thread_id)
     // MEP data
     auto& slice = m_net_slices[i_buffer];
 
-    // Fill blocks
-    MEP::find_blocks(slice.mep, slice.blocks);
-
-    // Fill fragment offsets
-    MEP::fragment_offsets(slice.blocks, slice.offsets);
-
     // Transpose or calculate offsets
     if (m_transpose_mep.value()) {
       // Transpose the events into the slice
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index fed2cdd8e..6766cdf6f 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -174,7 +174,7 @@ private:
     std::condition_variable& cond,
     std::unique_lock<std::mutex>& lock);
 
-  void set_intervals(BufferStatus& intervals, size_t n_events);
+  void prepare_mep(size_t i_buffer, size_t n_events);
 
   // mep reader thread
   void mep_read();
-- 
GitLab


From c9316eef60ecf859c758e352ae08d236c372e274 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 21 Apr 2022 16:28:09 +0200
Subject: [PATCH 088/120] Stricter check on packing factors

---
 AllenOnline/src/MEPProvider.cpp | 52 ++++++++++++++-------------------
 AllenOnline/src/MEPProvider.h   |  4 ++-
 2 files changed, 25 insertions(+), 31 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index a19b885c7..e3997489e 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -706,12 +706,7 @@ bool MEPProvider::allocate_storage(size_t i_read)
   auto const* mep = slice.mep;
   size_t n_blocks = mep->header.n_MFPs;
 
-  if (m_packing_factor == 0) {
-    m_packing_factor = slice.packing_factor;
-  }
-  else {
-    assert(slice.packing_factor == m_packing_factor);
-  }
+  m_packing_factor = slice.packing_factor;
 
   size_t const eps = m_events_per_slice.value();
   auto n_interval = m_packing_factor / eps;
@@ -841,17 +836,23 @@ std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvi
   return {it, distance(m_buffer_status.begin(), it)};
 }
 
-void MEPProvider::prepare_mep(size_t i_buffer, size_t n_events)
+bool MEPProvider::prepare_mep(size_t i_buffer, size_t n_events)
 {
   auto& slice = m_net_slices[i_buffer];
 
+  if (slice.packing_factor > m_packing_factor) {
+    error() << "MEP encountered with packing factor larger than first MEP." << endmsg;
+    read_error();
+    return false;
+  }
+
   // Fill blocks
   MEP::find_blocks(slice.mep, slice.blocks);
 
   // Fill fragment offsets
   MEP::fragment_offsets(slice.blocks, slice.offsets);
 
-  if (n_events == 0) return;
+  if (n_events == 0) return true;
 
   auto& status = m_buffer_status[i_buffer];
   assert(status.work_counter == 0);
@@ -874,6 +875,16 @@ void MEPProvider::prepare_mep(size_t i_buffer, size_t n_events)
     intervals.emplace_back((i - 1) * eps, i * eps);
   }
   status.work_counter = intervals.size();
+
+  return true;
+}
+
+void MEPProvider::read_error()
+{
+  m_read_error = true;
+  m_transpose_cond.notify_all();
+  m_receive_cond.notify_all();
+  m_transposed_cond.notify_one();
 }
 
 // mep reader thread
@@ -884,12 +895,6 @@ void MEPProvider::mep_read()
   size_t preloaded = 0;
   std::vector<bool> preloaded_buffer(n_buffers(), false);
 
-  auto read_error = [this] {
-    m_read_error = true;
-    m_transpose_cond.notify_all();
-    m_transposed_cond.notify_one();
-  };
-
   auto to_read = this->n_events();
   if (to_read && msgLevel(MSG::DEBUG)) debug() << "Reading " << *to_read << " events" << endmsg;
   auto to_publish = 0;
@@ -1014,7 +1019,6 @@ void MEPProvider::mep_read()
       read_error();
       break;
     }
-    assert(slice.packing_factor <= m_packing_factor);
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
@@ -1034,7 +1038,7 @@ void MEPProvider::mep_read()
           (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
 
           // Set intervals for offset calculation/transposition
-          prepare_mep(i_buffer, n_events);
+          if (!prepare_mep(i_buffer, n_events)) break;
         }
         else {
           // We didn't read anything, so free the buffer we got again
@@ -1059,11 +1063,6 @@ void MEPProvider::mpi_read()
 {
   #ifdef HAVE_MPI
 
-  auto read_error = [this] {
-    m_read_error = true;
-    m_transpose_cond.notify_all();
-  };
-
   std::vector<MPI_Request> requests(m_window_size);
 
   // Iterate over the slices
@@ -1275,7 +1274,7 @@ void MEPProvider::mpi_read()
     if (!mpi_error) {
       {
         std::unique_lock<std::mutex> lock {m_buffer_mutex};
-        prepare_mep(i_buffer, size_t {slice.packing_factor});
+        if (!prepare_mep(i_buffer, size_t {slice.packing_factor})) break;
       }
       if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
       m_transpose_cond.notify_one();
@@ -1296,13 +1295,6 @@ void MEPProvider::mpi_read()
 // buffer manager reader thread
 void MEPProvider::bm_read(const std::string& buffer_name)
 {
-  auto read_error = [this] {
-    m_read_error = true;
-    m_transpose_cond.notify_all();
-    m_receive_cond.notify_all();
-    m_control_cond.notify_all();
-  };
-
   auto const partitionID = m_allenConfig->partitionID.value();
   auto to_read = this->n_events();
   if (to_read)
@@ -1424,7 +1416,7 @@ void MEPProvider::bm_read(const std::string& buffer_name)
     // ready. If prefetching is done, wake up all threads
     if (!cancelled) {
       std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      prepare_mep(i_buffer, size_t {slice.packing_factor});
+      if (!prepare_mep(i_buffer, size_t {slice.packing_factor})) break;
     }
 
     if (!cancelled) {
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index 6766cdf6f..8b038b992 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -174,7 +174,9 @@ private:
     std::condition_variable& cond,
     std::unique_lock<std::mutex>& lock);
 
-  void prepare_mep(size_t i_buffer, size_t n_events);
+  bool prepare_mep(size_t i_buffer, size_t n_events);
+
+  void read_error();
 
   // mep reader thread
   void mep_read();
-- 
GitLab


From 15893938864dd1a43dbd047726090afa7c4a7f6e Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 22 Apr 2022 09:27:19 +0200
Subject: [PATCH 089/120] Fix MEPProvider stop if no MEP were received

---
 AllenOnline/src/MEPProvider.cpp | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index e3997489e..5bc00ca6a 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -448,6 +448,9 @@ StatusCode MEPProvider::stop()
       }
     }
 
+    // Notify input threads in case they're waiting and there hasn't been any data.
+    m_control_cond.notify_all();
+
     for (auto& input_thread : m_input_threads) {
       input_thread.join();
     }
-- 
GitLab


From 96ebe7d2cba3b44cdcac8c8d3996ab3f3d87b6f8 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 22 Apr 2022 09:27:40 +0200
Subject: [PATCH 090/120] Rename Allen.py to AllenConfig.py to avoid module
 name clashes

---
 .../options/{Allen.py => AllenConfig.py}      | 30 ++++++++++---------
 1 file changed, 16 insertions(+), 14 deletions(-)
 rename AllenOnline/options/{Allen.py => AllenConfig.py} (91%)

diff --git a/AllenOnline/options/Allen.py b/AllenOnline/options/AllenConfig.py
similarity index 91%
rename from AllenOnline/options/Allen.py
rename to AllenOnline/options/AllenConfig.py
index 5ec64ba1e..292f49b9c 100755
--- a/AllenOnline/options/Allen.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -42,12 +42,15 @@ integration_test = False
 app = LHCbApp(
     DataType="Upgrade",
     Simulation=True,
-    DDDBtag="dddb-20210218",
-    CondDBtag="sim-20201218-vc-md100")
+    DDDBtag="dddb-20210617",
+    CondDBtag="sim-20210617-vc-md100")
+    # DDDBtag="dddb-20210218",
+    # CondDBtag="sim-20201218-vc-md100")
 
 setup_allen_non_event_data_service()
 
 appMgr = ApplicationMgr()
+appMgr.AppName = ""
 
 # Upgrade DBs
 CondDB().Upgrade = True
@@ -73,8 +76,6 @@ online_conf.logDeviceFormat = '%TIME%LEVEL%-8NODE: %-32PROCESS %-20SOURCE'
 online_conf.OutputLevel = 3
 online_conf.IOOutputLevel = output_level
 
-appMgr = ApplicationMgr()
-
 if run_online:
     import fifo_log
     online_conf.logDeviceType = 'fifo'
@@ -96,15 +97,15 @@ allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
 allen_conf.PartitionID = partition_id
 allen_conf.EnableRunChanges = True
-#allen_conf.Verbosity = 5
 
 if run_online:
     from Configurables import Allen__MBMOutput as MBMOutput
-    output_svc = MBMOutput()
-    output_svc.OutputLevel = 3
+    output_svc = MBMOutput("MBMOutput")
+    output_svc.OutputLevel = 2
     output_svc.Checksum = True
     output_svc.BufferName = mbm_setup.Allen_Output
     appMgr.ExtSvc += [output_svc]
+    allen_conf.Output = "mbm://" + output_svc.getFullName()
 elif integration_test:
     allen_conf.Output = "tcp://192.168.1.101:35000"
 else:
@@ -112,7 +113,8 @@ else:
 
 mep_provider = MEPProvider()
 mep_provider.NSlices = 36
-mep_provider.EventsPerSlice = 10000
+# Events per slice should be set, and memory reserved per thread as well
+mep_provider.EventsPerSlice = 1000
 mep_provider.OutputLevel = 3
 # Number of MEP buffers and number of transpose/offset threads
 mep_provider.BufferConfig = (12, 8)
@@ -148,17 +150,17 @@ monSvc.PartitionName = partition
 monSvc.DimUpdateInterval = 5
 
 appMgr.MessageSvcType = 'MessageSvc'
-appMgr.OutputLevel = output_level
+appMgr.OutputLevel = 3
 messageSvc = MessageSvc('MessageSvc')
 messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
-messageSvc.OutputLevel = output_level
+messageSvc.OutputLevel = 3
 
 
 # Add the services that will produce the non-event-data
-# appMgr.ExtSvc = [
-#     monSvc,
-#     "OnlMonitorSink"
-# ] + appMgr.ExtSvc
+appMgr.ExtSvc = [
+    monSvc,
+    "OnlMonitorSink"
+] + appMgr.ExtSvc
 
 appMgr.EvtSel = "NONE"
 
-- 
GitLab


From 8935539408f3c3473972af2f936aff0a32cee556 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 22 Apr 2022 09:28:12 +0200
Subject: [PATCH 091/120] Make initialization of MBM output handling less
 fragile

---
 AllenOnline/src/AllenApplication.cpp | 22 +++++++---------------
 AllenOnline/src/AllenApplication.h   |  2 +-
 2 files changed, 8 insertions(+), 16 deletions(-)

diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 853917c7e..be81973d3 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -343,11 +343,8 @@ OutputHandler* AllenApplication::makeOutput()
     m_logger->warning("No lines configured in Allen sequence");
   }
 
-  SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
-  auto outputSvc = sloc->service("Allen::MBMOutput/MBMOutput", false);
-
   auto output = m_allenConfig->output.value();
-  if (!outputSvc && output.empty()) {
+  if (output.empty()) {
     m_logger->warning("No output is configured, selected events will be lost");
     return {};
   }
@@ -371,23 +368,18 @@ OutputHandler* AllenApplication::makeOutput()
     m_outputHolder = Allen::output_handler(m_provider.get(), m_zmqSvc.get(), std::move(options));
     return m_outputHolder.get();
   }
-  else if (outputSvc) {
-    if (m_allenConfig->partitionBuffers.value()) {
-      connection += "_";
-      auto const partition = m_allenConfig->partition.value();
-      std::stringstream hexID;
-      hexID << std::hex << m_allenConfig->partitionID.value();
-      connection += partition.empty() ? hexID.str() : m_allenConfig->partition.value();
-    }
+  else if (output_type == "mbm") {
+    SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
+    auto outputSvc = sloc->service(connection.substr(p + 3, std::string::npos), false);
 
     if (!outputSvc) {
       m_logger->error("Failed to retrieve MBMOutput.");
-      return {};
+      return nullptr;
     }
 
-    auto* mbmOutput = dynamic_cast<Allen::MBMOutput*>(outputSvc.get());
+    auto* mbmOutput = dynamic_cast<OutputHandler*>(outputSvc.get());
     if (mbmOutput == nullptr) {
-      m_logger->error("Failed to cast MBMOutput");
+      m_logger->error("Failed to cast MBMOutput to OutputHandler");
     }
     return mbmOutput;
   }
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 6f6baf170..2409997f1 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -101,7 +101,7 @@ private:
 
   std::shared_ptr<IInputProvider> m_provider;
   std::unique_ptr<OutputHandler> m_outputHolder;
-  OutputHandler* m_output;
+  OutputHandler* m_output = nullptr;
 
   size_t m_nSlices = 0;
 
-- 
GitLab


From 33e10c5b3475a0a62cf37e7b0c7bbb032873c0bb Mon Sep 17 00:00:00 2001
From: Gitlab CI <noreply@cern.ch>
Date: Fri, 22 Apr 2022 07:34:10 +0000
Subject: [PATCH 092/120] Fixed formatting

patch generated by https://gitlab.cern.ch/lhcb/MooreOnline/-/jobs/21259894
---
 AllenOnline/application/MPISend.cpp           |  182 +--
 AllenOnline/application/read_mep.cpp          |  204 ++-
 .../application/test_host_register.cpp        |   36 +-
 AllenOnline/include/AllenOnline/MPIConfig.h   |   22 +-
 AllenOnline/include/AllenOnline/ReadMEP.h     |    2 +-
 .../include/AllenOnline/TransposeMEP.h        |  126 +-
 AllenOnline/options/AllenConfig.py            |   27 +-
 AllenOnline/src/AllenApplication.cpp          |  313 ++--
 AllenOnline/src/AllenApplication.h            |   43 +-
 AllenOnline/src/AllenConfiguration.cpp        |   13 +-
 AllenOnline/src/AllenConfiguration.h          |   80 +-
 AllenOnline/src/IOMonitor.cpp                 |    5 +-
 AllenOnline/src/MBMOutput.cpp                 |  144 +-
 AllenOnline/src/MBMOutput.h                   |   75 +-
 AllenOnline/src/MEPProvider.cpp               | 1379 ++++++++---------
 AllenOnline/src/MEPProvider.h                 |  213 ++-
 AllenOnline/src/ReadMEP.cpp                   |   55 +-
 AllenOnline/src/TransposeMEP.cpp              |  209 ++-
 18 files changed, 1410 insertions(+), 1718 deletions(-)

diff --git a/AllenOnline/application/MPISend.cpp b/AllenOnline/application/MPISend.cpp
index 97d7bc24a..9b5e6b6b9 100644
--- a/AllenOnline/application/MPISend.cpp
+++ b/AllenOnline/application/MPISend.cpp
@@ -1,32 +1,30 @@
 /*****************************************************************************\
 * (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
 \*****************************************************************************/
+#include <iostream>
 #include <map>
+#include <optional>
 #include <string>
-#include <iostream>
 #include <vector>
-#include <optional>
 
-#include <sys/types.h>
-#include <sys/stat.h>
 #include <fcntl.h>
+#include <sys/stat.h>
+#include <sys/types.h>
 #include <unistd.h>
 
-#include <boost/program_options.hpp>
 #include <boost/algorithm/string.hpp>
+#include <boost/program_options.hpp>
 
-#include <AllenOnline/ReadMEP.h>
 #include <AllenOnline/MPIConfig.h>
+#include <AllenOnline/ReadMEP.h>
 
 namespace MPI {
   int rank;
 
-  std::string rank_str()
-  {
-    if (rank == receiver) {
+  std::string rank_str() {
+    if ( rank == receiver ) {
       return "MPI::Receiver: ";
-    }
-    else {
+    } else {
       return "MPI::Sender: ";
     }
   }
@@ -35,33 +33,30 @@ namespace MPI {
 namespace {
   namespace po = boost::program_options;
   namespace ba = boost::algorithm;
-}
+} // namespace
 
-int main(int argc, char* argv[]) {
+int main( int argc, char* argv[] ) {
 
   std::string mep_input;
-  size_t window_size = 4;
-  bool non_stop = false;
-  size_t number_of_meps = 0;
-
-  po::options_description desc("Allowed options");
-  desc.add_options()
-    ("help", "produce help message")
-    ("mep", po::value<std::string>(&mep_input),
-     "mep files as comma-separated list")
-    ("mpi-window-size", po::value<size_t>(&window_size)->default_value(4),
-     "MPI sliding window size")
-    ("non-stop", po::value<bool>(&non_stop)->default_value(false), "input file")
-    ("n-meps,n", po::value<size_t>(&number_of_meps)->default_value(0), "#MEPs");
+  size_t      window_size    = 4;
+  bool        non_stop       = false;
+  size_t      number_of_meps = 0;
+
+  po::options_description desc( "Allowed options" );
+  desc.add_options()( "help", "produce help message" )( "mep", po::value<std::string>( &mep_input ),
+                                                        "mep files as comma-separated list" )(
+      "mpi-window-size", po::value<size_t>( &window_size )->default_value( 4 ),
+      "MPI sliding window size" )( "non-stop", po::value<bool>( &non_stop )->default_value( false ), "input file" )(
+      "n-meps,n", po::value<size_t>( &number_of_meps )->default_value( 0 ), "#MEPs" );
 
   po::positional_options_description p;
-  p.add("mep", 1);
+  p.add( "mep", 1 );
 
   po::variables_map vm;
-  po::store(po::command_line_parser(argc, argv).options(desc).positional(p).run(), vm);
-  po::notify(vm);
+  po::store( po::command_line_parser( argc, argv ).options( desc ).positional( p ).run(), vm );
+  po::notify( vm );
 
-  if (vm.count("help")) {
+  if ( vm.count( "help" ) ) {
     std::cout << desc << "\n";
     return 1;
   }
@@ -69,64 +64,60 @@ int main(int argc, char* argv[]) {
   // The sender is in charge of reading all MDF files and passing
   // them to the receiver.
 
-  if (!vm.count("mep")) {
+  if ( !vm.count( "mep" ) ) {
     std::cout << MPI::rank_str() << "Required argument --mep not supplied. Exiting application.\n";
     return -1;
   }
 
-#if defined(HAVE_MPI)
+#if defined( HAVE_MPI )
   std::vector<std::string> connections;
-  ba::split(connections, mep_input, ba::is_any_of(","));
+  ba::split( connections, mep_input, ba::is_any_of( "," ) );
 
   // Create requests of appropiate size
-  std::vector<MPI_Request> requests(window_size);
+  std::vector<MPI_Request> requests( window_size );
 
   // Read all files in connections
   std::vector<std::tuple<EB::MEP const*, gsl::span<char const>>> meps;
 
   std::cout << MPI::rank_str() << "Reading "
-            << (number_of_meps != 0 ? std::to_string(number_of_meps) : std::string {"all"}) << " meps from files\n";
+            << ( number_of_meps != 0 ? std::to_string( number_of_meps ) : std::string{"all"} ) << " meps from files\n";
 
-  std::vector<char> data;
+  std::vector<char>     data;
   gsl::span<char const> mep_span;
-  size_t n_meps_read = 0;
+  size_t                n_meps_read = 0;
 
   std::optional<unsigned> packing_factor = std::nullopt;
 
-  for (const auto& connection : connections) {
-    bool eof = false, success = true;
+  for ( const auto& connection : connections ) {
+    bool           eof = false, success = true;
     EB::MEP const* mep = nullptr;
-    unsigned pf = 0;
+    unsigned       pf  = 0;
 
-    auto input = LHCb::StreamDescriptor::bind(connection);
-    if (input.ioDesc != 0) {
-      std::cout << "Opened " << connection << "\n";
-    }
+    auto input = LHCb::StreamDescriptor::bind( connection );
+    if ( input.ioDesc != 0 ) { std::cout << "Opened " << connection << "\n"; }
 
-    while (success && !eof) {
+    while ( success && !eof ) {
       std::cout << "." << std::flush;
 
-      std::tie(eof, success, mep, pf, mep_span) = MEP::read_mep(input, data);
-      if (!packing_factor) {
+      std::tie( eof, success, mep, pf, mep_span ) = MEP::read_mep( input, data );
+      if ( !packing_factor ) {
         packing_factor = pf;
-      }
-      else if (*packing_factor != pf) {
-        std::cout << "Got MEP with different packing factor: " << pf
-                  << " instead of: " << *packing_factor << "\n";
+      } else if ( *packing_factor != pf ) {
+        std::cout << "Got MEP with different packing factor: " << pf << " instead of: " << *packing_factor << "\n";
         return 1;
       }
 
-      if (!eof && success) {
+      if ( !eof && success ) {
         char* contents = nullptr;
-        MPI_Alloc_mem(mep_span.size(), MPI_INFO_NULL, &contents);
+        MPI_Alloc_mem( mep_span.size(), MPI_INFO_NULL, &contents );
 
         // Populate contents with stream buf
-        std::copy_n(mep_span.data(), mep_span.size(), contents);
+        std::copy_n( mep_span.data(), mep_span.size(), contents );
         ++n_meps_read;
 
-        meps.emplace_back(mep, gsl::span<char> {contents, mep_span.size()});
+        meps.emplace_back( mep, gsl::span<char>{contents, mep_span.size()} );
       }
-      if (n_meps_read >= number_of_meps && number_of_meps != 0) {
+      if ( n_meps_read >= number_of_meps && number_of_meps != 0 ) {
         input.close();
         goto send;
       }
@@ -136,46 +127,46 @@ int main(int argc, char* argv[]) {
 
 send:
 
-  if (meps.empty() || !packing_factor) {
+  if ( meps.empty() || !packing_factor ) {
     std::cout << "Failed to read MEPs from file\n";
     return 1;
   }
 
   // MPI initialization
-  MPI_Init(&argc, &argv);
+  MPI_Init( &argc, &argv );
 
   // Communication size
   int comm_size;
-  MPI_Comm_size(MPI_COMM_WORLD, &comm_size);
-  if (comm_size > MPI::comm_size) {
+  MPI_Comm_size( MPI_COMM_WORLD, &comm_size );
+  if ( comm_size > MPI::comm_size ) {
     std::cout << "This program requires at most " << MPI::comm_size << " processes.\n";
     return -1;
   }
 
   // MPI: Who am I?
-  MPI_Comm_rank(MPI_COMM_WORLD, &MPI::rank);
+  MPI_Comm_rank( MPI_COMM_WORLD, &MPI::rank );
 
-  auto const& first_mep = std::get<0>(*meps.begin());
+  auto const& first_mep = std::get<0>( *meps.begin() );
 
   size_t pf = *packing_factor;
   std::cout << "\n"
-            << MPI::rank_str() << "MEP header: " << first_mep->header.n_MFPs << ", " << pf << ", "
-            << first_mep->bytes() << "\n";
-  MPI_Send(&pf, 1, MPI_SIZE_T, MPI::receiver, MPI::message::packing_factor, MPI_COMM_WORLD);
+            << MPI::rank_str() << "MEP header: " << first_mep->header.n_MFPs << ", " << pf << ", " << first_mep->bytes()
+            << "\n";
+  MPI_Send( &pf, 1, MPI_SIZE_T, MPI::receiver, MPI::message::packing_factor, MPI_COMM_WORLD );
 
-  MPI_Send(&n_meps_read, 1, MPI_SIZE_T, MPI::receiver, MPI::message::number_of_meps, MPI_COMM_WORLD);
+  MPI_Send( &n_meps_read, 1, MPI_SIZE_T, MPI::receiver, MPI::message::number_of_meps, MPI_COMM_WORLD );
 
   // Test: Send all the files
   size_t current_mep = 0;
-  while (non_stop || current_mep < meps.size()) {
+  while ( non_stop || current_mep < meps.size() ) {
 
     // Get event data
-    auto const [mep, mep_span] = meps[current_mep];
-    const char* current_event_start = mep_span.data();
-    const size_t current_event_size = mep_span.size_bytes();
+    auto const [mep, mep_span]       = meps[current_mep];
+    const char*  current_event_start = mep_span.data();
+    const size_t current_event_size  = mep_span.size_bytes();
 
     // Notify the event size
-    MPI_Send(&current_event_size, 1, MPI_SIZE_T, MPI::receiver, MPI::message::event_size, MPI_COMM_WORLD);
+    MPI_Send( &current_event_size, 1, MPI_SIZE_T, MPI::receiver, MPI::message::event_size, MPI_COMM_WORLD );
 
     // Number of full-size (MPI::mdf_chunk_size) messages
     size_t n_messages = current_event_size / MPI::mdf_chunk_size;
@@ -185,52 +176,33 @@ send:
     size_t n_sends = n_messages > window_size ? window_size : n_messages;
 
     // Initial parallel sends
-    for (size_t k = 0; k < n_sends; k++) {
+    for ( size_t k = 0; k < n_sends; k++ ) {
       const char* message = current_event_start + k * MPI::mdf_chunk_size;
-      MPI_Isend(
-        message,
-        MPI::mdf_chunk_size,
-        MPI_BYTE,
-        MPI::receiver,
-        MPI::message::event_send_tag_start + k,
-        MPI_COMM_WORLD,
-        &requests[k]);
+      MPI_Isend( message, MPI::mdf_chunk_size, MPI_BYTE, MPI::receiver, MPI::message::event_send_tag_start + k,
+                 MPI_COMM_WORLD, &requests[k] );
     }
     // Sliding window sends
-    for (size_t k = n_sends; k < n_messages; k++) {
+    for ( size_t k = n_sends; k < n_messages; k++ ) {
       int r;
-      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      MPI_Waitany( window_size, requests.data(), &r, MPI_STATUS_IGNORE );
       const char* message = current_event_start + k * MPI::mdf_chunk_size;
-      MPI_Isend(
-        message,
-        MPI::mdf_chunk_size,
-        MPI_BYTE,
-        MPI::receiver,
-        MPI::message::event_send_tag_start + k,
-        MPI_COMM_WORLD,
-        &requests[r]);
+      MPI_Isend( message, MPI::mdf_chunk_size, MPI_BYTE, MPI::receiver, MPI::message::event_send_tag_start + k,
+                 MPI_COMM_WORLD, &requests[r] );
     }
     // Last send (if necessary)
-    if (rest) {
+    if ( rest ) {
       int r;
-      MPI_Waitany(window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      MPI_Waitany( window_size, requests.data(), &r, MPI_STATUS_IGNORE );
       const char* message = current_event_start + n_messages * MPI::mdf_chunk_size;
-      MPI_Isend(
-        message,
-        rest,
-        MPI_BYTE,
-        MPI::receiver,
-        MPI::message::event_send_tag_start + n_messages,
-        MPI_COMM_WORLD,
-        &requests[r]);
+      MPI_Isend( message, rest, MPI_BYTE, MPI::receiver, MPI::message::event_send_tag_start + n_messages,
+                 MPI_COMM_WORLD, &requests[r] );
     }
     // Wait until all chunks have been sent
-    MPI_Waitall(n_sends, requests.data(), MPI_STATUSES_IGNORE);
+    MPI_Waitall( n_sends, requests.data(), MPI_STATUSES_IGNORE );
 
-    if (non_stop) {
-      current_mep = (current_mep + 1) % meps.size();
-    }
-    else {
+    if ( non_stop ) {
+      current_mep = ( current_mep + 1 ) % meps.size();
+    } else {
       ++current_mep;
     }
   }
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 25235902a..3916aa36a 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -1,108 +1,105 @@
 /*****************************************************************************\
 * (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
 \*****************************************************************************/
+#include <cassert>
 #include <cstring>
-#include <iostream>
 #include <fstream>
-#include <string>
 #include <iomanip>
-#include <unordered_set>
+#include <iostream>
 #include <map>
-#include <cassert>
+#include <string>
+#include <unordered_set>
 
-#include <sys/types.h>
-#include <sys/stat.h>
 #include <fcntl.h>
+#include <sys/stat.h>
+#include <sys/types.h>
 #include <unistd.h>
 
-#include <Event/RawBank.h>
 #include <Event/ODIN.h>
+#include <Event/RawBank.h>
 
 #include <Allen/Logger.h>
-#include <Allen/SliceUtils.h>
-#include <Allen/sourceid.h>
 #include <Allen/MEPTools.h>
+#include <Allen/SliceUtils.h>
 #include <Allen/read_mdf.hpp>
+#include <Allen/sourceid.h>
 
 #include <MDF/StreamDescriptor.h>
 
 #include <EventBuilding/MEP_tools.hpp>
 #include <EventBuilding/MFP_tools.hpp>
 
-#include <AllenOnline/TransposeMEP.h>
 #include <AllenOnline/ReadMEP.h>
+#include <AllenOnline/TransposeMEP.h>
 
 using namespace std;
 
 struct CaloRawBank {
-  uint32_t source_id = 0;
-  uint32_t const* data = nullptr;
-  uint32_t const* end = nullptr;
+  uint32_t        source_id = 0;
+  uint32_t const* data      = nullptr;
+  uint32_t const* end       = nullptr;
 
   // For MEP format
-  CaloRawBank(const uint32_t sid, const char* fragment, const uint16_t s) :
-    source_id {sid}, data {reinterpret_cast<uint32_t const*>(fragment)}, end {reinterpret_cast<uint32_t const*>(fragment + s)}
-  {
-    assert(s % sizeof(uint32_t) == 0);
+  CaloRawBank( const uint32_t sid, const char* fragment, const uint16_t s )
+      : source_id{sid}
+      , data{reinterpret_cast<uint32_t const*>( fragment )}
+      , end{reinterpret_cast<uint32_t const*>( fragment + s )} {
+    assert( s % sizeof( uint32_t ) == 0 );
   }
 };
 
 struct ODINRawBank {
 
   uint32_t const* data = nullptr;
-  uint16_t size = 0;
+  uint16_t        size = 0;
 
   /// Constructor from MEP layout
-  ODINRawBank(const uint32_t, const char* fragment, uint16_t s)
-  {
-    data = reinterpret_cast<uint32_t const*>(fragment);
+  ODINRawBank( const uint32_t, const char* fragment, uint16_t s ) {
+    data = reinterpret_cast<uint32_t const*>( fragment );
     size = s;
   }
 };
 
-std::vector<char> contiguous_mfps(Allen::Slice const& mep_data)
-{
+std::vector<char> contiguous_mfps( Allen::Slice const& mep_data ) {
   // To make direct use of the offsets, the MFPs need to be copied
   // into temporary storage
-  auto const& mfps = mep_data.fragments;
-  vector<char> mep_fragments(mep_data.fragments_mem_size, '\0');
-  char* destination = &mep_fragments[0];
-  for (gsl::span<char const> mfp : mfps) {
-    ::memcpy(destination, mfp.data(), mfp.size_bytes());
+  auto const&  mfps = mep_data.fragments;
+  vector<char> mep_fragments( mep_data.fragments_mem_size, '\0' );
+  char*        destination = &mep_fragments[0];
+  for ( gsl::span<char const> mfp : mfps ) {
+    ::memcpy( destination, mfp.data(), mfp.size_bytes() );
     destination += mfp.size_bytes();
   }
   return mep_fragments;
 }
 
-int main(int argc, char* argv[])
-{
-  if (argc != 3) {
+int main( int argc, char* argv[] ) {
+  if ( argc != 3 ) {
     cout << "usage: read_mep file.mep n_mep" << endl;
     return -1;
   }
 
   string filename = {argv[1]};
-  size_t n_meps = atol(argv[2]);
+  size_t n_meps   = atol( argv[2] );
 
   // Some storage for reading the events into
   bool eof = false, success = false;
 
-  auto input = LHCb::StreamDescriptor::bind(filename);
-  if (input.ioDesc != 0) {
+  auto input = LHCb::StreamDescriptor::bind( filename );
+  if ( input.ioDesc != 0 ) {
     info_cout << "Opened " << filename << "\n";
-  }
-  else {
-    cerr << "Failed to open file " << filename << " " << strerror(errno) << "\n";
+  } else {
+    cerr << "Failed to open file " << filename << " " << strerror( errno ) << "\n";
     return -1;
   }
 
   vector<char> data;
 
-  EventIDs event_ids;
+  EventIDs          event_ids;
   std::vector<char> event_mask;
 
-  MEP::Slices mep_slices(1);
-  auto& slice = mep_slices[0];
+  MEP::Slices mep_slices( 1 );
+  auto&       slice = mep_slices[0];
 
   Allen::Slices bank_slices;
 
@@ -110,53 +107,42 @@ int main(int argc, char* argv[])
 
   // 160 bytes per fragment, 60 fragments
   std::vector<char> calo_banks;
-  calo_banks.resize(9600);
+  calo_banks.resize( 9600 );
 
-  for (size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep) {
+  for ( size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep ) {
 
-    std::tie(eof, success, slice.mep, slice.packing_factor, slice.mep_data) = MEP::read_mep(input, data);
-    auto const* mep = slice.mep;
-    if (!success) {
+    std::tie( eof, success, slice.mep, slice.packing_factor, slice.mep_data ) = MEP::read_mep( input, data );
+    auto const* mep                                                           = slice.mep;
+    if ( !success ) {
       return 1;
-    }
-    else {
+    } else {
       cout << "Read mep with packing factor " << slice.packing_factor << " #MFPs: " << mep->header.n_MFPs << "\n";
     }
 
-    if (i_mep == 0) {
-      event_ids.reserve(slice.packing_factor);
-      event_mask.resize(slice.packing_factor, 0);
-      slice.blocks.resize(mep->header.n_MFPs, MEP::Blocks::value_type{});
-      slice.offsets.resize(mep->header.n_MFPs);
-      for (auto& offsets : slice.offsets) {
-        offsets.resize(slice.packing_factor + 1);
-      }
+    if ( i_mep == 0 ) {
+      event_ids.reserve( slice.packing_factor );
+      event_mask.resize( slice.packing_factor, 0 );
+      slice.blocks.resize( mep->header.n_MFPs, MEP::Blocks::value_type{} );
+      slice.offsets.resize( mep->header.n_MFPs );
+      for ( auto& offsets : slice.offsets ) { offsets.resize( slice.packing_factor + 1 ); }
     }
 
-    MEP::find_blocks(mep, slice.blocks);
-    auto [success, mfp_count, banks_version] = MEP::fill_counts(mep);
-
-    if (i_mep == 0) {
-      bank_slices = allocate_slices(1, bank_types,
-        [pf = slice.packing_factor, &mc = mfp_count](auto bt) -> std::tuple<size_t, size_t, size_t> {
-          auto n_blocks = mc[to_integral(bt)];
-          auto n_sizes = pf * ((n_blocks + 1) / 2 + 1);
-          return {0ul, n_sizes, 2 + n_blocks + (1 + pf) * (1 + n_blocks) - 2};
-        });
+    MEP::find_blocks( mep, slice.blocks );
+    auto [success, mfp_count, banks_version] = MEP::fill_counts( mep );
+
+    if ( i_mep == 0 ) {
+      bank_slices = allocate_slices(
+          1, bank_types, [pf = slice.packing_factor, &mc = mfp_count]( auto bt ) -> std::tuple<size_t, size_t, size_t> {
+            auto n_blocks = mc[to_integral( bt )];
+            auto n_sizes  = pf * ( ( n_blocks + 1 ) / 2 + 1 );
+            return {0ul, n_sizes, 2 + n_blocks + ( 1 + pf ) * ( 1 + n_blocks ) - 2};
+          } );
+    } else {
+      reset_slice( bank_slices, 0, bank_types, event_ids, true );
     }
-    else {
-      reset_slice(bank_slices, 0, bank_types, event_ids, true);
-    }
-
-    MEP::mep_offsets(bank_slices, 0,
-                     bank_types,
-                     mfp_count,
-                     event_ids,
-                     event_mask,
-                     slice.blocks,
-                     {0ul, slice.packing_factor},
-                     false);
 
+    MEP::mep_offsets( bank_slices, 0, bank_types, mfp_count, event_ids, event_mask, slice.blocks,
+                      {0ul, slice.packing_factor}, false );
 
     // In MEP layout the fragmets are split into MFPs that are not
     // contiguous in memory. When the data is copied to the device the
@@ -164,50 +150,48 @@ int main(int argc, char* argv[])
     // contiguous; the offsets are prepared with this in mind.
 
     auto const& first_block = slice.blocks.front();
-    cout << "MEP with packing: " << std::setw(4) << first_block.header->n_banks
-         << " event_id: " << std::setw(6) << first_block.header->ev_id << "\n";
+    cout << "MEP with packing: " << std::setw( 4 ) << first_block.header->n_banks << " event_id: " << std::setw( 6 )
+         << first_block.header->ev_id << "\n";
 
     // Decode first ODIN
-    auto const odin_index = to_integral(BankTypes::ODIN);
-    auto const& odin_slice = bank_slices[odin_index][0];
-    auto odin_banks = contiguous_mfps(odin_slice);
+    auto const  odin_index   = to_integral( BankTypes::ODIN );
+    auto const& odin_slice   = bank_slices[odin_index][0];
+    auto        odin_banks   = contiguous_mfps( odin_slice );
     auto const& odin_offsets = odin_slice.offsets;
-    auto const& odin_sizes = odin_slice.sizes;
-    auto const& odin_types = odin_slice.types;
-    auto odin_bank = MEP::raw_bank<ODINRawBank>(odin_banks.data(), odin_offsets.data(), odin_sizes.data(), odin_types.data(), 0, 0);
-    LHCb::ODIN odin;
-    if (banks_version[odin_index] == 7) {
+    auto const& odin_sizes   = odin_slice.sizes;
+    auto const& odin_types   = odin_slice.types;
+    auto        odin_bank    = MEP::raw_bank<ODINRawBank>( odin_banks.data(), odin_offsets.data(), odin_sizes.data(),
+                                                 odin_types.data(), 0, 0 );
+    LHCb::ODIN  odin;
+    if ( banks_version[odin_index] == 7 ) {
       odin = LHCb::ODIN{{odin_bank.data, 10}};
+    } else {
+      odin = LHCb::ODIN::from_version<6>( {odin_bank.data, 10} );
     }
-    else {
-      odin = LHCb::ODIN::from_version<6>({odin_bank.data, 10});
-    }
-    cout << "ODIN version: " << banks_version[odin_index]
-         << " run: " << std::setw(7) << odin.runNumber()
-         << " event: " << std::setw(12) << odin.eventNumber() << "\n";
+    cout << "ODIN version: " << banks_version[odin_index] << " run: " << std::setw( 7 ) << odin.runNumber()
+         << " event: " << std::setw( 12 ) << odin.eventNumber() << "\n";
 
     // Print block information
-    for (auto const& block : slice.blocks) {
+    for ( auto const& block : slice.blocks ) {
       // block offsets are in number of 4-byte words
-      auto const source_id = block.header->src_id;
-      std::string det = SourceId_sysstr(source_id);
-      std::string fill(7 - det.size(), ' ');
-
-      size_t padded_size = 0;
-      auto const* bank_sizes = block.bank_sizes;
-      for (size_t i = 0; i < block.header->n_banks; ++i) {
-        padded_size += bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << block.header->align);
+      auto const  source_id = block.header->src_id;
+      std::string det       = SourceId_sysstr( source_id );
+      std::string fill( 7 - det.size(), ' ' );
+
+      size_t      padded_size = 0;
+      auto const* bank_sizes  = block.bank_sizes;
+      for ( size_t i = 0; i < block.header->n_banks; ++i ) {
+        padded_size += bank_sizes[i] + EB::get_padding( bank_sizes[i], 1 << block.header->align );
       }
 
       cout << "fragment"
-           << " magic: 0x" << std::hex << block.header->magic << std::dec
-           << " source_id: " << std::setw(6) << source_id
-           << " top5: " << std::setw(2) << SourceId_sys(source_id) << fill << " (" << det << ") "
-           << std::setw(5) << SourceId_num(source_id)
-           << " version: " << std::setw(2) << unsigned{block.header->block_version}
-           << " align: " << std::setw(3) << pow(2, unsigned{block.header->align})
-           << " size: " << std::setw(8) << block.header->bytes() - block.header->header_size()
-           << " padded_size: " << std::setw(8) << padded_size << "\n";
+           << " magic: 0x" << std::hex << block.header->magic << std::dec << " source_id: " << std::setw( 6 )
+           << source_id << " top5: " << std::setw( 2 ) << SourceId_sys( source_id ) << fill << " (" << det << ") "
+           << std::setw( 5 ) << SourceId_num( source_id ) << " version: " << std::setw( 2 )
+           << unsigned{block.header->block_version} << " align: " << std::setw( 3 )
+           << pow( 2, unsigned{block.header->align} ) << " size: " << std::setw( 8 )
+           << block.header->bytes() - block.header->header_size() << " padded_size: " << std::setw( 8 ) << padded_size
+           << "\n";
     }
   }
   return 0;
diff --git a/AllenOnline/application/test_host_register.cpp b/AllenOnline/application/test_host_register.cpp
index 73be9c8b6..a42a12a76 100644
--- a/AllenOnline/application/test_host_register.cpp
+++ b/AllenOnline/application/test_host_register.cpp
@@ -9,53 +9,51 @@
 * or submit itself to any jurisdiction.                                       *
 \*****************************************************************************/
 #include <Backend/BackendCommon.h>
-#include <RTL/rtl.h>
 #include <MBM/bmdef.h>
+#include <RTL/rtl.h>
 
-#include <iostream>
-#include <thread>
 #include <chrono>
 #include <iomanip>
+#include <iostream>
+#include <thread>
 
 int main() {
 
   const std::string buffer_name = "Events_0_TDET";
-  auto pn = RTL::processName() + ".0";
-  BMID bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), 32768, BM_COM_FIFO);
+  auto              pn          = RTL::processName() + ".0";
+  BMID              bmid        = ::mbm_include_read( buffer_name.c_str(), pn.c_str(), 32768, BM_COM_FIFO );
 
-  if (bmid == MBM_INV_DESC) {
+  if ( bmid == MBM_INV_DESC ) {
     std::cerr << "MBM: Failed to connect to MBM buffer " << buffer_name << std::endl;
     return -1;
   }
 
   // register buffer manager memory with the device runtime
-  size_t buffer_size = 0;
-  char* buffer_address = const_cast<char*>(::mbm_buffer_address(bmid));
-  ::mbm_buffer_size(bmid, &buffer_size);
+  size_t buffer_size    = 0;
+  char*  buffer_address = const_cast<char*>( ::mbm_buffer_address( bmid ) );
+  ::mbm_buffer_size( bmid, &buffer_size );
 
-  std::cout << "MBM buffer: " << reinterpret_cast<void*>(buffer_address) << " " << buffer_size << "\n";
+  std::cout << "MBM buffer: " << reinterpret_cast<void*>( buffer_address ) << " " << buffer_size << "\n";
 
   bool runtime_error = false;
   try {
-    Allen::set_device(0, 0);
-  } catch (const std::invalid_argument& e) {
+    Allen::set_device( 0, 0 );
+  } catch ( const std::invalid_argument& e ) {
     std::cerr << "Failed to set device: " << e.what() << "\n";
     runtime_error = true;
   }
 
-  if (!runtime_error) {
+  if ( !runtime_error ) {
     try {
-      Allen::host_register(buffer_address, buffer_size, Allen::hostRegisterReadOnly);
+      Allen::host_register( buffer_address, buffer_size, Allen::hostRegisterReadOnly );
       std::cout << "Successfully registered\n";
-    } catch (const std::invalid_argument& e) {
-      std::cerr << "Failed to register: " << e.what() << "\n";
-    }
+    } catch ( const std::invalid_argument& e ) { std::cerr << "Failed to register: " << e.what() << "\n"; }
   }
 
   using namespace std::chrono_literals;
-  std::this_thread::sleep_for(5s);
+  std::this_thread::sleep_for( 5s );
 
-  ::mbm_exclude(bmid);
+  ::mbm_exclude( bmid );
 
   return 0;
 }
diff --git a/AllenOnline/include/AllenOnline/MPIConfig.h b/AllenOnline/include/AllenOnline/MPIConfig.h
index 42de41445..4fbe5ef3a 100644
--- a/AllenOnline/include/AllenOnline/MPIConfig.h
+++ b/AllenOnline/include/AllenOnline/MPIConfig.h
@@ -3,22 +3,22 @@
 \*****************************************************************************/
 #pragma once
 
-#include <mpi.h>
 #include <limits.h>
+#include <mpi.h>
 
 // Determine size of size_t for MPI type
 #if SIZE_MAX == UCHAR_MAX
-#define MPI_SIZE_T MPI_UNSIGNED_CHAR
+#  define MPI_SIZE_T MPI_UNSIGNED_CHAR
 #elif SIZE_MAX == USHRT_MAX
-#define MPI_SIZE_T MPI_UNSIGNED_SHORT
+#  define MPI_SIZE_T MPI_UNSIGNED_SHORT
 #elif SIZE_MAX == UINT_MAX
-#define MPI_SIZE_T MPI_UNSIGNED
+#  define MPI_SIZE_T MPI_UNSIGNED
 #elif SIZE_MAX == ULONG_MAX
-#define MPI_SIZE_T MPI_UNSIGNED_LONG
+#  define MPI_SIZE_T MPI_UNSIGNED_LONG
 #elif SIZE_MAX == ULLONG_MAX
-#define MPI_SIZE_T MPI_UNSIGNED_LONG_LONG
+#  define MPI_SIZE_T MPI_UNSIGNED_LONG_LONG
 #else
-#error "size_t size could not be determined."
+#  error "size_t size could not be determined."
 #endif
 
 namespace MPI {
@@ -38,12 +38,12 @@ namespace MPI {
 
   namespace message {
     // Message tags
-    constexpr int test = 1;
+    constexpr int test           = 1;
     constexpr int packing_factor = 2;
     constexpr int number_of_meps = 3;
-    constexpr int max_file_size = 4;
-    constexpr int event_size = 5;
-    constexpr int window_size = 6;
+    constexpr int max_file_size  = 4;
+    constexpr int event_size     = 5;
+    constexpr int window_size    = 6;
 
     // Event sends will start with tag start, and cycle every modulo
     // ie:
diff --git a/AllenOnline/include/AllenOnline/ReadMEP.h b/AllenOnline/include/AllenOnline/ReadMEP.h
index 03bd59bce..8fa79db8a 100644
--- a/AllenOnline/include/AllenOnline/ReadMEP.h
+++ b/AllenOnline/include/AllenOnline/ReadMEP.h
@@ -12,5 +12,5 @@
 
 namespace MEP {
   std::tuple<bool, bool, EB::MEP const*, unsigned, gsl::span<char const>>
-  read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer);
+  read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer );
 }
diff --git a/AllenOnline/include/AllenOnline/TransposeMEP.h b/AllenOnline/include/AllenOnline/TransposeMEP.h
index 0c6cd0d22..f609d2cfd 100644
--- a/AllenOnline/include/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/include/AllenOnline/TransposeMEP.h
@@ -3,30 +3,30 @@
 \*****************************************************************************/
 #pragma once
 
-#include <thread>
-#include <vector>
+#include <algorithm>
 #include <array>
-#include <deque>
-#include <mutex>
 #include <atomic>
 #include <chrono>
-#include <algorithm>
-#include <numeric>
 #include <condition_variable>
+#include <deque>
+#include <mutex>
+#include <numeric>
+#include <thread>
+#include <vector>
 
-#include <unistd.h>
-#include <sys/types.h>
-#include <sys/stat.h>
 #include <fcntl.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+#include <unistd.h>
 
-#include <Allen/Common.h>
 #include <Allen/AllenUnits.h>
-#include <Allen/TransposeTypes.h>
 #include <Allen/BankMapping.h>
+#include <Allen/Common.h>
+#include <Allen/TransposeTypes.h>
 #include <Event/RawBank.h>
 
-#include <EventBuilding/MFP_tools.hpp>
 #include <EventBuilding/MEP_tools.hpp>
+#include <EventBuilding/MFP_tools.hpp>
 
 namespace {
   using namespace Allen::Units;
@@ -38,24 +38,29 @@ namespace MEP {
 
   struct Block {
     Block() = default;
-    Block(EB::MFP const* b) : mfp{b}, header{&(mfp->header)}, bank_sizes{mfp->header.bank_sizes()}, bank_types{mfp->header.bank_types()}, payload{static_cast<char const*>(mfp->payload())} {}
-
-    EB::MFP const* mfp = nullptr;
-    EB::MFP_header const* header = nullptr;
+    Block( EB::MFP const* b )
+        : mfp{b}
+        , header{&( mfp->header )}
+        , bank_sizes{mfp->header.bank_sizes()}
+        , bank_types{mfp->header.bank_types()}
+        , payload{static_cast<char const*>( mfp->payload() )} {}
+
+    EB::MFP const*            mfp        = nullptr;
+    EB::MFP_header const*     header     = nullptr;
     EB::bank_size_type const* bank_sizes = nullptr;
     EB::bank_type_type const* bank_types = nullptr;
-    char const* payload = nullptr;
+    char const*               payload    = nullptr;
   };
 
   using Blocks = std::vector<Block>;
 
   struct Slice {
-    EB::MEP const* mep = nullptr;
+    EB::MEP const*        mep = nullptr;
     gsl::span<char const> mep_data;
-    unsigned packing_factor = 0u;
-    Blocks blocks;
-    SourceOffsets offsets;
-    size_t slice_size = 0u;
+    unsigned              packing_factor = 0u;
+    Blocks                blocks;
+    SourceOffsets         offsets;
+    size_t                slice_size = 0u;
   };
   using Slices = std::vector<MEP::Slice>;
 
@@ -70,33 +75,23 @@ namespace MEP {
    *
    * @return     (success, number of banks per bank type; 0 if the bank is not needed, bank version per type)
    */
-  std::tuple<bool, std::array<unsigned int, NBankTypes>, std::array<int, NBankTypes>>
-  fill_counts(EB::MEP const* mep);
-
-  void find_blocks(EB::MEP const* mep, Blocks& blocks);
-
-  void fragment_offsets(Blocks const& blocks, std::vector<std::vector<uint32_t>>& offsets);
-
-  size_t allen_offsets(
-    Allen::Slices& slices,
-    int const slice_index,
-    std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, NBankTypes> const& mfp_count,
-    Blocks const& blocks,
-    MEP::SourceOffsets const& input_offsets,
-    std::tuple<size_t, size_t> const& interval,
-    bool split_by_run = false);
-
-  std::tuple<bool, bool, size_t> mep_offsets(
-    Allen::Slices& slices,
-    int const slice_index,
-    std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, NBankTypes> const& banks_count,
-    EventIDs& event_ids,
-    std::vector<char>& event_mask,
-    MEP::Blocks const& blocks,
-    std::tuple<size_t, size_t> const& interval,
-    bool split_by_run = false);
+  std::tuple<bool, std::array<unsigned int, NBankTypes>, std::array<int, NBankTypes>> fill_counts( EB::MEP const* mep );
+
+  void find_blocks( EB::MEP const* mep, Blocks& blocks );
+
+  void fragment_offsets( Blocks const& blocks, std::vector<std::vector<uint32_t>>& offsets );
+
+  size_t allen_offsets( Allen::Slices& slices, int const slice_index, std::unordered_set<BankTypes> const& bank_types,
+                        std::array<unsigned int, NBankTypes> const& mfp_count, Blocks const& blocks,
+                        MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval,
+                        bool split_by_run = false );
+
+  std::tuple<bool, bool, size_t> mep_offsets( Allen::Slices& slices, int const slice_index,
+                                              std::unordered_set<BankTypes> const&        bank_types,
+                                              std::array<unsigned int, NBankTypes> const& banks_count,
+                                              EventIDs& event_ids, std::vector<char>& event_mask,
+                                              MEP::Blocks const& blocks, std::tuple<size_t, size_t> const& interval,
+                                              bool split_by_run = false );
 
   /**
    * @brief      Transpose events to Allen layout
@@ -109,17 +104,10 @@ namespace MEP {
    *
    * @return     tuple of: (success, slice is full)
    */
-  bool transpose_mep(
-    Allen::Slices& slices,
-    int const slice_index,
-    std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, NBankTypes > const& mfp_count,
-    EventIDs& event_ids,
-    std::vector<char>& event_mask,
-    EB::MEP const* mep,
-    Blocks const& blocks,
-    SourceOffsets const& input_offsets,
-    std::tuple<size_t, size_t> const& interval);
+  bool transpose_mep( Allen::Slices& slices, int const slice_index, std::unordered_set<BankTypes> const& bank_types,
+                      std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
+                      std::vector<char>& event_mask, EB::MEP const* mep, Blocks const& blocks,
+                      SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval );
 
   /**
    * @brief      Transpose MEP to Allen layout
@@ -132,16 +120,10 @@ namespace MEP {
    *
    * @return     tuple of: (success, slice is full, number of events transposed)
    */
-  std::tuple<bool, bool, size_t> transpose_events(
-    Allen::Slices& slices,
-    int const slice_index,
-    std::unordered_set<BankTypes> const& bank_types,
-    std::array<unsigned int, NBankTypes> const& mfp_count,
-    EventIDs& event_ids,
-    std::vector<char>& event_mask,
-    EB::MEP const* mep,
-    Blocks const& blocks,
-    SourceOffsets const& source_offsets,
-    std::tuple<size_t, size_t> const& interval,
-    bool split_by_run = false);
+  std::tuple<bool, bool, size_t>
+  transpose_events( Allen::Slices& slices, int const slice_index, std::unordered_set<BankTypes> const& bank_types,
+                    std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
+                    std::vector<char>& event_mask, EB::MEP const* mep, Blocks const& blocks,
+                    SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval,
+                    bool split_by_run = false );
 } // namespace MEP
diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 292f49b9c..6478b7db7 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -44,8 +44,8 @@ app = LHCbApp(
     Simulation=True,
     DDDBtag="dddb-20210617",
     CondDBtag="sim-20210617-vc-md100")
-    # DDDBtag="dddb-20210218",
-    # CondDBtag="sim-20201218-vc-md100")
+# DDDBtag="dddb-20210218",
+# CondDBtag="sim-20201218-vc-md100")
 
 setup_allen_non_event_data_service()
 
@@ -59,7 +59,8 @@ CondDB().Upgrade = True
 # be done internally to Allen and retrieved somehow
 n_lines = 0
 # json_file = "/daqarea1/fest/allen/configuration/passthrough.json"
-json_file = os.path.expandvars("${ALLEN_INSTALL_DIR}/constants/passthrough.json")
+json_file = os.path.expandvars(
+    "${ALLEN_INSTALL_DIR}/constants/passthrough.json")
 with open(json_file) as jf:
     json_config = json.load(jf)
     gs = json_config.get('gather_selections', {})
@@ -125,7 +126,7 @@ mep_provider.ThreadPerBuffer = True
 if run_online:
     mep_provider.Source = "MBM"
     mep_provider.MBMComMethod = "FIFO"
-    mep_provider.Connections = mbm_setup.BU_Buffers #  ["Events_0", "Events_1"]
+    mep_provider.Connections = mbm_setup.BU_Buffers  #  ["Events_0", "Events_1"]
     mep_provider.Requests = [
         'EvType=1;TriggerMask=0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF,0xFFFFFFFF;VetoMask=0,0,0,0;MaskType=ANY;UserType=ONE;Frequency=PERC;Perc=100.0'
     ]
@@ -136,9 +137,10 @@ elif integration_test:
 else:
     mep_provider.Source = "Files"
     mep_dir = "/daqarea1/fest/mep"
-    mep_provider.Connections = sorted([os.path.join(mep_dir, mep_file)
-                                       for mep_file in os.listdir(mep_dir)
-                                       if mep_file.endswith('.mep')])
+    mep_provider.Connections = sorted([
+        os.path.join(mep_dir, mep_file) for mep_file in os.listdir(mep_dir)
+        if mep_file.endswith('.mep')
+    ])
     mep_provider.LoopOnMEPs = False
 
 monSvc = MonitorSvc('MonitorSvc')
@@ -155,18 +157,13 @@ messageSvc = MessageSvc('MessageSvc')
 messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
 messageSvc.OutputLevel = 3
 
-
 # Add the services that will produce the non-event-data
-appMgr.ExtSvc = [
-    monSvc,
-    "OnlMonitorSink"
-] + appMgr.ExtSvc
+appMgr.ExtSvc = [monSvc, "OnlMonitorSink"] + appMgr.ExtSvc
 
 appMgr.EvtSel = "NONE"
 
 # Key services, order matters!
 appMgr.ExtSvc = [
-    'ToolSvc', 'AuditorSvc', allen_conf,
-    'Online::Configuration/Application', 'ZeroMQSvc',
-    'MEPProvider'
+    'ToolSvc', 'AuditorSvc', allen_conf, 'Online::Configuration/Application',
+    'ZeroMQSvc', 'MEPProvider'
 ] + appMgr.ExtSvc
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index be81973d3..cc2c725d2 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -10,44 +10,44 @@
 \*****************************************************************************/
 #include <dlfcn.h>
 
-#include <iostream>
 #include <chrono>
-#include <thread>
 #include <cmath>
+#include <iostream>
 #include <regex>
+#include <thread>
 #include <unordered_set>
 
 #include <filesystem>
 
-#include <GaudiKernel/IMessageSvc.h>
+#include <Gaudi/Property.h>
+#include <GaudiKernel/AppReturnCode.h>
 #include <GaudiKernel/IAppMgrUI.h>
+#include <GaudiKernel/IMessageSvc.h>
+#include <GaudiKernel/IMonitorSvc.h>
 #include <GaudiKernel/IProperty.h>
 #include <GaudiKernel/ISvcLocator.h>
-#include <GaudiKernel/AppReturnCode.h>
-#include <Gaudi/Property.h>
 #include <GaudiKernel/SmartIF.h>
-#include <GaudiKernel/IMonitorSvc.h>
 
 #include <CPP/Event.h>
-#include <RTL/strdef.h>
 #include <RTL/rtl.h>
+#include <RTL/strdef.h>
 #include <dim/dis.hxx>
 
 #include <GaudiOnline/OnlineApplication.h>
 
-#include <Allen/BankTypes.h>
-#include <Allen/BankMapping.h>
 #include <Allen/Allen.h>
+#include <Allen/BankMapping.h>
+#include <Allen/BankTypes.h>
 #include <Allen/Provider.h>
 
 #include "MEPProvider.h"
 
 #ifdef HAVE_MPI
-#include <AllenOnline/MPIConfig.h>
+#  include <AllenOnline/MPIConfig.h>
 #endif
 
-#include "AllenConfiguration.h"
 #include "AllenApplication.h"
+#include "AllenConfiguration.h"
 #include "MBMOutput.h"
 
 // #include "EBProvider.h"
@@ -58,66 +58,61 @@ namespace {
 } // namespace
 
 /// Factory instantiation
-DECLARE_COMPONENT(AllenApplication)
+DECLARE_COMPONENT( AllenApplication )
 
 /// Specialized constructor
-AllenApplication::AllenApplication(Options opts) : OnlineApplication(opts) {}
+AllenApplication::AllenApplication( Options opts ) : OnlineApplication( opts ) {}
 
 // Default destructor
-AllenApplication::~AllenApplication()
-{
+AllenApplication::~AllenApplication() {
   // if (m_handle) {
   //   dlclose(m_handle);
   // }
 }
 
 /// Stop the application                             (RUNNING    -> READY)
-int AllenApplication::stop()
-{
-  m_zmqSvc->send(*m_allenControl, "STOP", zmq::send_flags::sndmore);
-  m_zmqSvc->send(*m_allenControl, m_allenConfig->stopTimeout.value());
+int AllenApplication::stop() {
+  m_zmqSvc->send( *m_allenControl, "STOP", zmq::send_flags::sndmore );
+  m_zmqSvc->send( *m_allenControl, m_allenConfig->stopTimeout.value() );
 
   // This will stop input
-  fireIncident("DAQ_STOP");
-
+  fireIncident( "DAQ_STOP" );
 
   StatusCode sc = app->stop();
-  if ( !sc.isSuccess() )   {
-    m_logger->error("%s Class %d application: Failed to execute the "
-                    "transition to state %s",
-                    "app->stop()", 1 , stateName(ST_READY));
-    fireIncident("DAQ_ERROR");
-    return declareState(ST_ERROR);
+  if ( !sc.isSuccess() ) {
+    m_logger->error( "%s Class %d application: Failed to execute the "
+                     "transition to state %s",
+                     "app->stop()", 1, stateName( ST_READY ) );
+    fireIncident( "DAQ_ERROR" );
+    return declareState( ST_ERROR );
   }
 
   zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
-  m_zmqSvc->poll(&items[0], 1, -1);
-  if (items[0].revents & zmq::POLLIN) {
-    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
-    if (msg == "READY") {
-      m_logger->info("Allen event loop is stopped");
-    }
-    else {
-      m_logger->error("Allen event loop failed to stop");
+  m_zmqSvc->poll( &items[0], 1, -1 );
+  if ( items[0].revents & zmq::POLLIN ) {
+    auto msg = m_zmqSvc->receive<std::string>( *m_allenControl );
+    if ( msg == "READY" ) {
+      m_logger->info( "Allen event loop is stopped" );
+    } else {
+      m_logger->error( "Allen event loop failed to stop" );
       return Online::ONLINE_ERROR;
     }
   }
 
   m_provider->release_buffers();
 
-  fireIncident("DAQ_STOPPED");
-  fireIncident("APP_STOPPED");
-  return declareState(ST_READY);
+  fireIncident( "DAQ_STOPPED" );
+  fireIncident( "APP_STOPPED" );
+  return declareState( ST_READY );
 }
 
 /// Cancel the application: Cancel IO request/Event loop
 int AllenApplication::cancel() { return Online::ONLINE_OK; }
 
 /// Internal: Initialize the application            (NOT_READY  -> READY)
-int AllenApplication::configureApplication()
-{
+int AllenApplication::configureApplication() {
   int ret = OnlineApplication::configureApplication();
-  if (ret != Online::ONLINE_OK) return ret;
+  if ( ret != Online::ONLINE_OK ) return ret;
 
   // dlopen libAllenLib
   // m_handle = dlopen("libAllenLib.so", RTLD_LAZY);
@@ -139,118 +134,113 @@ int AllenApplication::configureApplication()
 
   SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
 
-  m_allenConfig = sloc->service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
-  if (!m_allenConfig) {
-    m_logger->throwError("Failed to retrieve AllenConfiguration.");
+  m_allenConfig = sloc->service( "AllenConfiguration/AllenConfiguration" ).as<AllenConfiguration>();
+  if ( !m_allenConfig ) {
+    m_logger->throwError( "Failed to retrieve AllenConfiguration." );
     return Online::ONLINE_ERROR;
   }
 
-  m_zmqSvc = sloc->service<IZeroMQSvc>("ZeroMQSvc");
-  if (!m_zmqSvc) {
-    m_logger->error("Failed to retrieve IZeroMQSvc.");
+  m_zmqSvc = sloc->service<IZeroMQSvc>( "ZeroMQSvc" );
+  if ( !m_zmqSvc ) {
+    m_logger->error( "Failed to retrieve IZeroMQSvc." );
     return Online::ONLINE_ERROR;
   }
 
-  SmartIF<IService> updater = sloc->service<IService>("AllenUpdater");
-  if (!updater.get()) {
-    m_logger->error("Failed to retrieve AllenUpdater.");
+  SmartIF<IService> updater = sloc->service<IService>( "AllenUpdater" );
+  if ( !updater.get() ) {
+    m_logger->error( "Failed to retrieve AllenUpdater." );
     return Online::ONLINE_ERROR;
   }
-  m_updater = dynamic_cast<Allen::NonEventData::IUpdater*>(updater.get());
-  if (m_updater == nullptr) {
-    m_logger->error("Failed to cast AllenUpdater");
+  m_updater = dynamic_cast<Allen::NonEventData::IUpdater*>( updater.get() );
+  if ( m_updater == nullptr ) {
+    m_logger->error( "Failed to cast AllenUpdater" );
     return Online::ONLINE_ERROR;
   }
 
-  SmartIF<IService> mepProvider = sloc->service<IService>("MEPProvider");
-  if (!mepProvider.get()) {
-    m_logger->error("Failed to retrieve MEPProvider.");
+  SmartIF<IService> mepProvider = sloc->service<IService>( "MEPProvider" );
+  if ( !mepProvider.get() ) {
+    m_logger->error( "Failed to retrieve MEPProvider." );
     return Online::ONLINE_ERROR;
   }
 
   std::string value;
-  auto sc = mepProvider.as<IProperty>()->getProperty("NSlices", value);
-  if (!sc.isSuccess()) {
-    m_logger->error("Failed to get NSlices property from MEPProvider");
+  auto        sc = mepProvider.as<IProperty>()->getProperty( "NSlices", value );
+  if ( !sc.isSuccess() ) {
+    m_logger->error( "Failed to get NSlices property from MEPProvider" );
     return Online::ONLINE_ERROR;
-  }
-  else {
+  } else {
     using Gaudi::Parsers::parse;
-    sc = parse(m_nSlices, value);
-    if (!sc.isSuccess()) {
-      m_logger->error("Failed to parse NSlices property");
+    sc = parse( m_nSlices, value );
+    if ( !sc.isSuccess() ) {
+      m_logger->error( "Failed to parse NSlices property" );
       return Online::ONLINE_ERROR;
     }
   }
 
-  if (!m_config->monitorType.empty()) {
-    m_monSvc = sloc->service<IGauchoMonitorSvc>(m_config->monitorType);
-    if (!m_monSvc) {
-      m_logger->error("Cannot access monitoring service of type %s.", m_config->monitorType.c_str());
+  if ( !m_config->monitorType.empty() ) {
+    m_monSvc = sloc->service<IGauchoMonitorSvc>( m_config->monitorType );
+    if ( !m_monSvc ) {
+      m_logger->error( "Cannot access monitoring service of type %s.", m_config->monitorType.c_str() );
       return Online::ONLINE_ERROR;
     }
   }
 
-  auto* provider = dynamic_cast<IInputProvider*>(mepProvider.get());
-  if (provider == nullptr) {
-    m_logger->error("Failed to cast MEPProvider");
+  auto* provider = dynamic_cast<IInputProvider*>( mepProvider.get() );
+  if ( provider == nullptr ) {
+    m_logger->error( "Failed to cast MEPProvider" );
     return Online::ONLINE_ERROR;
   }
 
   // Use a deleter that does nothing to allow wrapping it in a shared pointer.
-  m_provider.reset(provider, [](IInputProvider*) {});
+  m_provider.reset( provider, []( IInputProvider* ) {} );
 
-  auto json = m_allenConfig->json;
-  auto paramDir = resolveEnvVars(m_allenConfig->paramDir);
+  auto json     = m_allenConfig->json;
+  auto paramDir = resolveEnvVars( m_allenConfig->paramDir );
 
-  m_options = {{"v", std::to_string(6 - m_config->outputLevel())},
+  m_options = {{"v", std::to_string( 6 - m_config->outputLevel() )},
                {"t", m_allenConfig->nThreads.toString()},
                {"params", paramDir},
                {"device", m_allenConfig->device.value()},
-               {"s", std::to_string(m_nSlices)},
-               {"disable-run-changes", std::to_string(!m_allenConfig->runChanges.value())},
+               {"s", std::to_string( m_nSlices )},
+               {"disable-run-changes", std::to_string( !m_allenConfig->runChanges.value() )},
                {"sequence", json},
                {"run-from-json", "1"}};
 
   m_output = makeOutput();
-  if (!m_allenConfig->output.value().empty() && !m_output) {
-    m_logger->error("Failed to create output for "s + m_allenConfig->output.value());
+  if ( !m_allenConfig->output.value().empty() && !m_output ) {
+    m_logger->error( "Failed to create output for "s + m_allenConfig->output.value() );
     return Online::ONLINE_ERROR;
   }
 
-  m_allenControl = m_zmqSvc->socket(zmq::PAIR);
-  m_allenControl->bind(m_controlConnection.c_str());
+  m_allenControl = m_zmqSvc->socket( zmq::PAIR );
+  m_allenControl->bind( m_controlConnection.c_str() );
 
-  m_allenThread = std::thread {&AllenApplication::allenLoop, this};
+  m_allenThread = std::thread{&AllenApplication::allenLoop, this};
 
   zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
-  m_zmqSvc->poll(&items[0], 1, -1);
-  if (items[0].revents & zmq::POLLIN) {
-    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
-    if (msg == "READY") {
-      m_logger->info("Allen event loop is ready");
-    }
+  m_zmqSvc->poll( &items[0], 1, -1 );
+  if ( items[0].revents & zmq::POLLIN ) {
+    auto msg = m_zmqSvc->receive<std::string>( *m_allenControl );
+    if ( msg == "READY" ) { m_logger->info( "Allen event loop is ready" ); }
   }
 
   return ret;
 }
 
 /// Internal: Finalize the application              (READY      -> NOT_READY)
-int AllenApplication::finalizeApplication()
-{
-  m_zmqSvc->send(*m_allenControl, "RESET");
+int AllenApplication::finalizeApplication() {
+  m_zmqSvc->send( *m_allenControl, "RESET" );
 
   zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
-  m_zmqSvc->poll(&items[0], 1, -1);
-  if (items[0].revents & zmq::POLLIN) {
-    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
-    if (msg == "NOT_READY") {
-      m_logger->info("Allen event loop has exited");
+  m_zmqSvc->poll( &items[0], 1, -1 );
+  if ( items[0].revents & zmq::POLLIN ) {
+    auto msg = m_zmqSvc->receive<std::string>( *m_allenControl );
+    if ( msg == "NOT_READY" ) {
+      m_logger->info( "Allen event loop has exited" );
 
       m_allenThread.join();
-    }
-    else {
-      m_logger->error("Allen event loop failed to exit");
+    } else {
+      m_logger->error( "Allen event loop failed to exit" );
       return Online::ONLINE_ERROR;
     }
   }
@@ -259,133 +249,112 @@ int AllenApplication::finalizeApplication()
 }
 
 /// Internal: Start the application                 (READY      -> RUNNING)
-int AllenApplication::startApplication()
-{
+int AllenApplication::startApplication() {
   StatusCode sc = app->start();
-  if (!sc.isSuccess()) {
-    return Online::ONLINE_ERROR;
-  }
+  if ( !sc.isSuccess() ) { return Online::ONLINE_ERROR; }
 
-  m_zmqSvc->send(*m_allenControl, "START");
+  m_zmqSvc->send( *m_allenControl, "START" );
 
   zmq::pollitem_t items[] = {{*m_allenControl, 0, zmq::POLLIN, 0}};
-  m_zmqSvc->poll(&items[0], 1, -1);
-  if (items[0].revents & zmq::POLLIN) {
-    auto msg = m_zmqSvc->receive<std::string>(*m_allenControl);
-    if (msg == "RUNNING") {
-      m_logger->info("Allen event loop is running");
-    }
-    else {
-      m_logger->error("Allen event loop failed to start");
+  m_zmqSvc->poll( &items[0], 1, -1 );
+  if ( items[0].revents & zmq::POLLIN ) {
+    auto msg = m_zmqSvc->receive<std::string>( *m_allenControl );
+    if ( msg == "RUNNING" ) {
+      m_logger->info( "Allen event loop is running" );
+    } else {
+      m_logger->error( "Allen event loop failed to start" );
       return Online::ONLINE_ERROR;
     }
   }
 
-  fireIncident("DAQ_RUNNING");
-  fireIncident("APP_RUNNING");
+  fireIncident( "DAQ_RUNNING" );
+  fireIncident( "APP_RUNNING" );
   return Online::ONLINE_OK;
 }
 
 /// Pause the application                            (RUNNING    -> READY)
-int AllenApplication::pauseProcessing()
-{
-  m_logger->debug("Pause the application.");
+int AllenApplication::pauseProcessing() {
+  m_logger->debug( "Pause the application." );
   return OnlineApplication::pauseProcessing();
 }
 
 /// Continue the application                        (PAUSED -> RUNNING )
-int AllenApplication::continueProcessing()
-{
-  m_logger->debug("Resume application processing.");
+int AllenApplication::continueProcessing() {
+  m_logger->debug( "Resume application processing." );
   return OnlineApplication::continueProcessing();
 }
 
-void AllenApplication::allenLoop()
-{
+void AllenApplication::allenLoop() {
   //--events-per-slice 1000 --non-stop 1 --with-mpi $1:1 -c 0 -v 3 -t 8 -s 18 --output-file tcp://192.168.1.101:35000
   //--device 23:00.0
-  allen(m_options, this, m_provider, m_output, m_zmqSvc.get(), m_controlConnection);
+  allen( m_options, this, m_provider, m_output, m_zmqSvc.get(), m_controlConnection );
 }
 
-void AllenApplication::update(unsigned long run)
-{
+void AllenApplication::update( unsigned long run ) {
   // Monitoring run change
-  if (m_runNumber == 0) {
+  if ( m_runNumber == 0 ) {
     m_runNumber = run;
-    m_monSvc->setRunNo(run);
-  }
-  else if (run > m_runNumber) {
-    m_monSvc->update(m_runNumber).ignore();
-    m_monSvc->resetHistos(nullptr);
-    m_monSvc->setRunNo(run);
+    m_monSvc->setRunNo( run );
+  } else if ( run > m_runNumber ) {
+    m_monSvc->update( m_runNumber ).ignore();
+    m_monSvc->resetHistos( nullptr );
+    m_monSvc->setRunNo( run );
     m_runNumber = run;
   }
 
   // Update conditions by forwarding to the real updater
-  m_updater->update(run);
+  m_updater->update( run );
 }
 
-void AllenApplication::registerConsumer(std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c)
-{
-  m_updater->registerConsumer(id, std::move(c));
+void AllenApplication::registerConsumer( std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c ) {
+  m_updater->registerConsumer( id, std::move( c ) );
 }
 
-void AllenApplication::registerProducer(std::string const& id, Allen::NonEventData::Producer p)
-{
-  m_updater->registerProducer(id, std::move(p));
+void AllenApplication::registerProducer( std::string const& id, Allen::NonEventData::Producer p ) {
+  m_updater->registerProducer( id, std::move( p ) );
 }
 
-
-OutputHandler* AllenApplication::makeOutput()
-{
+OutputHandler* AllenApplication::makeOutput() {
   size_t n_lines = m_allenConfig->nLines.value();
-  if (n_lines == 0) {
-    m_logger->warning("No lines configured in Allen sequence");
-  }
+  if ( n_lines == 0 ) { m_logger->warning( "No lines configured in Allen sequence" ); }
 
   auto output = m_allenConfig->output.value();
-  if (output.empty()) {
-    m_logger->warning("No output is configured, selected events will be lost");
+  if ( output.empty() ) {
+    m_logger->warning( "No output is configured, selected events will be lost" );
     return {};
   }
 
   std::string connection;
-  auto p = output.find("://");
+  auto        p = output.find( "://" );
   std::string output_type;
 
-  if (!output.empty() && p == std::string::npos) {
+  if ( !output.empty() && p == std::string::npos ) {
     output_type = "file";
-    connection = output;
-  }
-  else if (!output.empty()) {
-    output_type = output.substr(0, p);
-    connection = output;
+    connection  = output;
+  } else if ( !output.empty() ) {
+    output_type = output.substr( 0, p );
+    connection  = output;
   }
 
-  if (output_type == "file" || output_type == "tcp") {
-    auto options = m_options;
+  if ( output_type == "file" || output_type == "tcp" ) {
+    auto options           = m_options;
     options["output-file"] = connection;
-    m_outputHolder = Allen::output_handler(m_provider.get(), m_zmqSvc.get(), std::move(options));
+    m_outputHolder         = Allen::output_handler( m_provider.get(), m_zmqSvc.get(), std::move( options ) );
     return m_outputHolder.get();
-  }
-  else if (output_type == "mbm") {
-    SmartIF<ISvcLocator> sloc = app.as<ISvcLocator>();
-    auto outputSvc = sloc->service(connection.substr(p + 3, std::string::npos), false);
+  } else if ( output_type == "mbm" ) {
+    SmartIF<ISvcLocator> sloc      = app.as<ISvcLocator>();
+    auto                 outputSvc = sloc->service( connection.substr( p + 3, std::string::npos ), false );
 
-    if (!outputSvc) {
-      m_logger->error("Failed to retrieve MBMOutput.");
+    if ( !outputSvc ) {
+      m_logger->error( "Failed to retrieve MBMOutput." );
       return nullptr;
     }
 
-    auto* mbmOutput = dynamic_cast<OutputHandler*>(outputSvc.get());
-    if (mbmOutput == nullptr) {
-      m_logger->error("Failed to cast MBMOutput to OutputHandler");
-    }
+    auto* mbmOutput = dynamic_cast<OutputHandler*>( outputSvc.get() );
+    if ( mbmOutput == nullptr ) { m_logger->error( "Failed to cast MBMOutput to OutputHandler" ); }
     return mbmOutput;
-  }
-  else {
-    m_logger->error("Unknown output type: "s + output_type);
+  } else {
+    m_logger->error( "Unknown output type: "s + output_type );
     return nullptr;
   }
-
 }
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 2409997f1..2fd15a647 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -10,37 +10,36 @@
 \*****************************************************************************/
 #include <dlfcn.h>
 
-#include <iostream>
 #include <chrono>
 #include <cmath>
+#include <iostream>
 
-#include <GaudiKernel/IMessageSvc.h>
+#include <CPP/Event.h>
+#include <Gaudi/Property.h>
+#include <GaudiKernel/AppReturnCode.h>
 #include <GaudiKernel/IAppMgrUI.h>
+#include <GaudiKernel/IMessageSvc.h>
+#include <GaudiKernel/IMonitorSvc.h>
 #include <GaudiKernel/IProperty.h>
 #include <GaudiKernel/ISvcLocator.h>
-#include <GaudiKernel/AppReturnCode.h>
-#include <Gaudi/Property.h>
 #include <GaudiKernel/SmartIF.h>
-#include <GaudiKernel/IMonitorSvc.h>
-#include <CPP/Event.h>
-#include <RTL/strdef.h>
 #include <RTL/rtl.h>
+#include <RTL/strdef.h>
 #include <dim/dis.hxx>
 
-#include <GaudiOnline/OnlineApplication.h>
 #include <Gaucho/IGauchoMonitorSvc.h>
+#include <GaudiOnline/OnlineApplication.h>
 #include <ZeroMQ/IZeroMQSvc.h>
 
-#include <Dumpers/IUpdater.h>
 #include <Allen/OutputHandler.h>
+#include <Dumpers/IUpdater.h>
 
 #include "AllenConfiguration.h"
 
 class AllenApplication : public Online::OnlineApplication, public Allen::NonEventData::IUpdater {
 public:
-
   // Specialized constructor
-  AllenApplication(Options opts);
+  AllenApplication( Options opts );
   // Default destructor
   virtual ~AllenApplication();
 
@@ -67,14 +66,13 @@ public:
   bool initMPI();
 
   // Updater
-  void update(unsigned long run) override;
+  void update( unsigned long run ) override;
 
-  void registerConsumer(std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c) override;
+  void registerConsumer( std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c ) override;
 
-  void registerProducer(std::string const& id, Allen::NonEventData::Producer p) override;
+  void registerProducer( std::string const& id, Allen::NonEventData::Producer p ) override;
 
 private:
-
   OutputHandler* makeOutput();
 
   void runChange();
@@ -93,24 +91,25 @@ private:
   unsigned long m_runNumber = 0;
 
   Allen::NonEventData::IUpdater* m_updater = nullptr;
-  SmartIF<AllenConfiguration> m_allenConfig;
+  SmartIF<AllenConfiguration>    m_allenConfig;
 
-  std::string const m_controlConnection = "inproc://AllenApplicationControl";
-  std::string const m_runChangeConnection = "inproc://AllenApplicationRunChange";
+  std::string const                  m_controlConnection   = "inproc://AllenApplicationControl";
+  std::string const                  m_runChangeConnection = "inproc://AllenApplicationRunChange";
   std::map<std::string, std::string> m_options;
 
   std::shared_ptr<IInputProvider> m_provider;
-  std::unique_ptr<OutputHandler> m_outputHolder;
-  OutputHandler* m_output = nullptr;
+  std::unique_ptr<OutputHandler>  m_outputHolder;
+  OutputHandler*                  m_output = nullptr;
 
   size_t m_nSlices = 0;
 
   // dlopen stuff to workaround segfault in genconf.exe
   // void* m_handle = nullptr;
   // typedef int (
-  //   *allen_t)(std::map<std::string, std::string>, Allen::NonEventData::IUpdater*, IZeroMQSvc* zmqSvc, std::string_view);
+  //   *allen_t)(std::map<std::string, std::string>, Allen::NonEventData::IUpdater*, IZeroMQSvc* zmqSvc,
+  //   std::string_view);
   // allen_t m_allenFun = nullptr;
 
-  std::thread m_allenThread;
+  std::thread                  m_allenThread;
   std::optional<zmq::socket_t> m_allenControl = std::nullopt;
 };
diff --git a/AllenOnline/src/AllenConfiguration.cpp b/AllenOnline/src/AllenConfiguration.cpp
index e14bfc415..c3dd3bde3 100644
--- a/AllenOnline/src/AllenConfiguration.cpp
+++ b/AllenOnline/src/AllenConfiguration.cpp
@@ -8,22 +8,21 @@
 * granted to it by virtue of its status as an Intergovernmental Organization  *
 * or submit itself to any jurisdiction.                                       *
 \*****************************************************************************/
-#include "GaudiKernel/Service.h"
 #include "AllenConfiguration.h"
+#include "GaudiKernel/Service.h"
 
-DECLARE_COMPONENT(AllenConfiguration)
+DECLARE_COMPONENT( AllenConfiguration )
 
 /// Query interfaces of Interface
-StatusCode AllenConfiguration::queryInterface(const InterfaceID& riid, void** ppv)
-{
-  if (AllenConfiguration::interfaceID().versionMatch(riid)) {
+StatusCode AllenConfiguration::queryInterface( const InterfaceID& riid, void** ppv ) {
+  if ( AllenConfiguration::interfaceID().versionMatch( riid ) ) {
     *ppv = this;
     addRef();
     return StatusCode::SUCCESS;
   }
-  return Service::queryInterface(riid, ppv);
+  return Service::queryInterface( riid, ppv );
 }
 
-AllenConfiguration::AllenConfiguration(std::string name, ISvcLocator* svcloc) : Service(name, svcloc) {}
+AllenConfiguration::AllenConfiguration( std::string name, ISvcLocator* svcloc ) : Service( name, svcloc ) {}
 
 AllenConfiguration::~AllenConfiguration() {}
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index 9f690c773..a3b04bd40 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -15,63 +15,63 @@
 #include <GaudiKernel/Service.h>
 
 namespace {
-  std::string resolveEnvVars(std::string s)
-  {
-    std::regex envExpr {"\\$\\{([A-Za-z0-9_]+)\\}"};
+  std::string resolveEnvVars( std::string s ) {
+    std::regex  envExpr{"\\$\\{([A-Za-z0-9_]+)\\}"};
     std::smatch m;
-    while (std::regex_search(s, m, envExpr)) {
+    while ( std::regex_search( s, m, envExpr ) ) {
       std::string rep;
-      System::getEnv(m[1].str(), rep);
-      s = s.replace(m[1].first - 2, m[1].second + 1, rep);
+      System::getEnv( m[1].str(), rep );
+      s = s.replace( m[1].first - 2, m[1].second + 1, rep );
     }
     return s;
   }
 
   namespace fs = std::filesystem;
-}
+} // namespace
 
 class AllenConfiguration : public Service {
 public:
   /// Retrieve interface ID
-  static const InterfaceID& interfaceID()
-  {
+  static const InterfaceID& interfaceID() {
     // Declaration of the interface ID.
-    static const InterfaceID iid("AllenConfiguration", 0, 0);
+    static const InterfaceID iid( "AllenConfiguration", 0, 0 );
     return iid;
   }
 
   /// Query interfaces of Interface
-  StatusCode queryInterface(const InterfaceID& riid, void** ppv) override;
-  AllenConfiguration(std::string name, ISvcLocator* svcloc);
+  StatusCode queryInterface( const InterfaceID& riid, void** ppv ) override;
+  AllenConfiguration( std::string name, ISvcLocator* svcloc );
 
   ~AllenConfiguration();
 
-  Gaudi::Property<float> stopTimeout {this, "StopTimeout", 5.};
-  Gaudi::Property<unsigned int> nThreads {this, "NThreads", 8};
-  Gaudi::Property<std::string> output {this, "Output", ""};
-  Gaudi::Property<bool> runChanges {this, "EnableRunChanges", true};
-  Gaudi::Property<std::string> device {this, "Device", "0"};
-  Gaudi::Property<std::string> json {this, "JSON", "${ALLEN_INSTALL_DIR}/constants/hlt1_pp_default.json",
-    [this](auto &) -> void {
-      auto const json_file = resolveEnvVars(json.value());
-      fs::path j{json_file};
-      if (!fs::exists(j) || !fs::is_regular_file(j)) {
-        throw GaudiException{"Allen JSON configuration file does not exist or is not a regular file", name(), StatusCode::FAILURE};
-      }
-      json.set(json_file);
-    }};
-  Gaudi::Property<size_t> nLines{this, "NLines", 0ul};
-  Gaudi::Property<std::string> paramDir {this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/parameters",
-    [this](auto &) -> void {
-      auto const dir = resolveEnvVars(paramDir.value());
-      fs::path p{dir};
-      if (!fs::exists(p) || !fs::is_directory(p)) {
-        throw GaudiException{"Allen parameter directory file does not exist or is not a directory", name(), StatusCode::FAILURE};
-      }
-      paramDir.set(dir);
-    }};
-  Gaudi::Property<bool> checksum {this, "OutputChecksum", false};
-  Gaudi::Property<unsigned> partitionID {this, "PartitionID", 0};
-  Gaudi::Property<bool> partitionBuffers {this, "PartitionBuffers", true};
-  Gaudi::Property<std::string> partition {this, "Partition", ""};
+  Gaudi::Property<float>        stopTimeout{this, "StopTimeout", 5.};
+  Gaudi::Property<unsigned int> nThreads{this, "NThreads", 8};
+  Gaudi::Property<std::string>  output{this, "Output", ""};
+  Gaudi::Property<bool>         runChanges{this, "EnableRunChanges", true};
+  Gaudi::Property<std::string>  device{this, "Device", "0"};
+  Gaudi::Property<std::string>  json{
+      this, "JSON", "${ALLEN_INSTALL_DIR}/constants/hlt1_pp_default.json", [this]( auto& ) -> void {
+        auto const json_file = resolveEnvVars( json.value() );
+        fs::path   j{json_file};
+        if ( !fs::exists( j ) || !fs::is_regular_file( j ) ) {
+          throw GaudiException{"Allen JSON configuration file does not exist or is not a regular file", name(),
+                               StatusCode::FAILURE};
+        }
+        json.set( json_file );
+      }};
+  Gaudi::Property<size_t>      nLines{this, "NLines", 0ul};
+  Gaudi::Property<std::string> paramDir{
+      this, "ParamDir", "${ALLEN_PROJECT_ROOT}/input/parameters", [this]( auto& ) -> void {
+        auto const dir = resolveEnvVars( paramDir.value() );
+        fs::path   p{dir};
+        if ( !fs::exists( p ) || !fs::is_directory( p ) ) {
+          throw GaudiException{"Allen parameter directory file does not exist or is not a directory", name(),
+                               StatusCode::FAILURE};
+        }
+        paramDir.set( dir );
+      }};
+  Gaudi::Property<bool>        checksum{this, "OutputChecksum", false};
+  Gaudi::Property<unsigned>    partitionID{this, "PartitionID", 0};
+  Gaudi::Property<bool>        partitionBuffers{this, "PartitionBuffers", true};
+  Gaudi::Property<std::string> partition{this, "Partition", ""};
 };
diff --git a/AllenOnline/src/IOMonitor.cpp b/AllenOnline/src/IOMonitor.cpp
index b22d60cf1..19aa8a296 100644
--- a/AllenOnline/src/IOMonitor.cpp
+++ b/AllenOnline/src/IOMonitor.cpp
@@ -7,8 +7,7 @@ namespace Allen {
   class IOMonitor : public Service {
 
     using Service::Service;
-
   };
-}
+} // namespace Allen
 
-DECLARE_COMPONENT_WITH_ID(Allen::IOMonitor, "AllenIOMon")
+DECLARE_COMPONENT_WITH_ID( Allen::IOMonitor, "AllenIOMon" )
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 3725bb205..3dfba6a76 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -15,28 +15,19 @@
 
 #include "MBMOutput.h"
 
-DECLARE_COMPONENT(Allen::MBMOutput)
+DECLARE_COMPONENT( Allen::MBMOutput )
 
-Allen::MBMOutput::MBMOutput(std::string name, ISvcLocator* loc)
-  : Service {name, loc}
-{
-}
+Allen::MBMOutput::MBMOutput( std::string name, ISvcLocator* loc ) : Service{name, loc} {}
 
-int Allen::MBMOutput::spaceCallback(void* /* param */)
-{
-  return MBM_NORMAL;
-}
+int Allen::MBMOutput::spaceCallback( void* /* param */ ) { return MBM_NORMAL; }
 
-std::tuple<size_t, gsl::span<char>> Allen::MBMOutput::buffer(size_t buffer_size, size_t n_events)
-{
-  if (m_cancelled) {
-    return {0, {}};
-  }
+std::tuple<size_t, gsl::span<char>> Allen::MBMOutput::buffer( size_t buffer_size, size_t n_events ) {
+  if ( m_cancelled ) { return {0, {}}; }
 
   {
     std::lock_guard<std::mutex> lock{m_mutex};
-    size_t id = 0;
-    if (m_freeIDs.empty()) {
+    size_t                      id = 0;
+    if ( m_freeIDs.empty() ) {
       id = m_id++;
     } else {
       id = m_freeIDs.front();
@@ -46,38 +37,37 @@ std::tuple<size_t, gsl::span<char>> Allen::MBMOutput::buffer(size_t buffer_size,
   }
 
   int* buf = nullptr;
-  auto sc = ::mbm_get_space_a(m_bmID, buffer_size, &buf, spaceCallback, this);
-  if ( sc == MBM_NORMAL )  {
-    sc = ::mbm_wait_space(m_bmID);
-    if ( sc == MBM_NORMAL )   {
-      m_buffer = {reinterpret_cast<char*>(buf), static_cast<events_size>(buffer_size)};
+  auto sc  = ::mbm_get_space_a( m_bmID, buffer_size, &buf, spaceCallback, this );
+  if ( sc == MBM_NORMAL ) {
+    sc = ::mbm_wait_space( m_bmID );
+    if ( sc == MBM_NORMAL ) {
+      m_buffer = {reinterpret_cast<char*>( buf ), static_cast<events_size>( buffer_size )};
       return {0, m_buffer};
     }
   } else {
-    error() << m_processName << " failed to get space of size " << buffer_size << " in buffer " << m_bufferName.value() << endmsg;
+    error() << m_processName << " failed to get space of size " << buffer_size << " in buffer " << m_bufferName.value()
+            << endmsg;
   }
   return {0, {}};
 }
 
-
-StatusCode Allen::MBMOutput::initialize()
-{
+StatusCode Allen::MBMOutput::initialize() {
   auto sc = Service::initialize();
-  if (!sc) return sc;
+  if ( !sc ) return sc;
 
-  m_allenConfig = serviceLocator()->service("AllenConfiguration/AllenConfiguration").as<AllenConfiguration>();
-  if (!m_allenConfig) {
+  m_allenConfig = serviceLocator()->service( "AllenConfiguration/AllenConfiguration" ).as<AllenConfiguration>();
+  if ( !m_allenConfig ) {
     error() << "Failed to retrieve AllenConfiguration." << endmsg;
     return StatusCode::FAILURE;
   }
 
-  SmartIF<IService> mepProvider = serviceLocator()->service<IService>("MEPProvider");
-  if (!mepProvider.get()) {
+  SmartIF<IService> mepProvider = serviceLocator()->service<IService>( "MEPProvider" );
+  if ( !mepProvider.get() ) {
     error() << "Failed to retrieve MEPProvider." << endmsg;
     return StatusCode::FAILURE;
   }
-  auto* inputProvider = dynamic_cast<IInputProvider*>(mepProvider.get());
-  if (inputProvider == nullptr) {
+  auto* inputProvider = dynamic_cast<IInputProvider*>( mepProvider.get() );
+  if ( inputProvider == nullptr ) {
     error() << "Failed to cast MEPProvider" << endmsg;
     return StatusCode::FAILURE;
   }
@@ -86,11 +76,11 @@ StatusCode Allen::MBMOutput::initialize()
 
   size_t nLines = m_allenConfig->nLines.value();
 
-  m_partitionID = m_allenConfig->partitionID.value();
-  std::string connection = m_bufferName.value();
-  auto const partition = m_allenConfig->partition.value();
-  bool partitionBuffers = m_allenConfig->partitionBuffers.value();
-  if (partitionBuffers) {
+  m_partitionID                = m_allenConfig->partitionID.value();
+  std::string connection       = m_bufferName.value();
+  auto const  partition        = m_allenConfig->partition.value();
+  bool        partitionBuffers = m_allenConfig->partitionBuffers.value();
+  if ( partitionBuffers ) {
     connection += "_";
     std::stringstream hexID;
     hexID << std::hex << m_partitionID;
@@ -99,77 +89,70 @@ StatusCode Allen::MBMOutput::initialize()
 
   info() << "Connect to MBM: " << connection << endmsg;
 
-  auto eventsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Events", true).get());
-  if (!eventsSvc) {
+  auto eventsSvc = dynamic_cast<Service*>( service<IService>( "AllenIOMon/Events", true ).get() );
+  if ( !eventsSvc ) {
     error() << "Failed to obtain Events service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
 
-  auto burstsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Bursts", true).get());
-  if (!eventsSvc) {
+  auto burstsSvc = dynamic_cast<Service*>( service<IService>( "AllenIOMon/Bursts", true ).get() );
+  if ( !eventsSvc ) {
     error() << "Failed to obtain Bursts service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
 
-  m_burstsOutput = std::make_unique<Gaudi::Accumulators::Counter<>>(burstsSvc, "OUT");
-  m_eventsOutput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "OUT");
-  m_mbOutput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "MB_OUT");
+  m_burstsOutput = std::make_unique<Gaudi::Accumulators::Counter<>>( burstsSvc, "OUT" );
+  m_eventsOutput = std::make_unique<Gaudi::Accumulators::Counter<>>( eventsSvc, "OUT" );
+  m_mbOutput     = std::make_unique<Gaudi::Accumulators::Counter<>>( eventsSvc, "MB_OUT" );
 
-  init(inputProvider, connection, nLines, m_checksum.value());
+  init( inputProvider, connection, nLines, m_checksum.value() );
   return StatusCode::SUCCESS;
 }
 
-StatusCode Allen::MBMOutput::start()
-{
+StatusCode Allen::MBMOutput::start() {
   auto sc = Service::start();
-  if (!sc) return sc;
+  if ( !sc ) return sc;
 
   m_cancelled = false;
-  m_bmID = ::mbm_include_write(connection().c_str(), m_processName.c_str(), m_partitionID, BM_COM_FIFO);
-  if (m_bmID == MBM_INV_DESC)   {
+  m_bmID      = ::mbm_include_write( connection().c_str(), m_processName.c_str(), m_partitionID, BM_COM_FIFO );
+  if ( m_bmID == MBM_INV_DESC ) {
     error() << "MBMOutput: failed to connect to MBM buffer " << connection() << endmsg;
     return StatusCode::FAILURE;
-    } else {
+  } else {
     info() << "MBMOutput: connected to MBM buffer " << connection() << endmsg;
     return StatusCode::SUCCESS;
   }
 }
 
-bool Allen::MBMOutput::write_buffer(size_t id)
-{
-  if (m_cancelled) {
-    return false;
-  }
+bool Allen::MBMOutput::write_buffer( size_t id ) {
+  if ( m_cancelled ) { return false; }
 
   size_t n_events = 0;
   {
     std::lock_guard<std::mutex> lock{m_mutex};
-    auto it = m_eventCounts.find(id);
-    n_events = it->second;
-    m_eventCounts.erase(it);
-    m_freeIDs.push_back(id);
+    auto                        it = m_eventCounts.find( id );
+    n_events                       = it->second;
+    m_eventCounts.erase( it );
+    m_freeIDs.push_back( id );
   }
 
-  (*m_burstsOutput) += 1;
-  (*m_eventsOutput) += n_events;
-  (*m_mbOutput) += (2 * m_buffer.size_bytes() + 1) / (2 * 1024 * 1024);
+  ( *m_burstsOutput ) += 1;
+  ( *m_eventsOutput ) += n_events;
+  ( *m_mbOutput ) += ( 2 * m_buffer.size_bytes() + 1 ) / ( 2 * 1024 * 1024 );
 
-  unsigned int mask[] = { m_partitionID, ~0x0U, ~0x0U, ~0x0U };
-  void* free_address = nullptr;
-  long free_len = 0;
+  unsigned int mask[]       = {m_partitionID, ~0x0U, ~0x0U, ~0x0U};
+  void*        free_address = nullptr;
+  long         free_len     = 0;
 
-  auto sc = ::mbm_declare_event(m_bmID, m_buffer.size_bytes(),
-                                EVENT_TYPE_EVENT,
-                                mask, 0, &free_address, &free_len, m_partitionID);
-  if (sc == MBM_REQ_CANCEL) {
+  auto sc = ::mbm_declare_event( m_bmID, m_buffer.size_bytes(), EVENT_TYPE_EVENT, mask, 0, &free_address, &free_len,
+                                 m_partitionID );
+  if ( sc == MBM_REQ_CANCEL ) {
     return false;
-  }
-  else if ( sc == MBM_NORMAL )  {
-    sc = ::mbm_send_space(m_bmID);
+  } else if ( sc == MBM_NORMAL ) {
+    sc = ::mbm_send_space( m_bmID );
     if ( sc == MBM_REQ_CANCEL ) {
       return false;
-    }
-    else {
+    } else {
       return true;
     }
   } else {
@@ -179,16 +162,11 @@ bool Allen::MBMOutput::write_buffer(size_t id)
 }
 
 void Allen::MBMOutput::cancel() {
-  if (!m_cancelled && m_bmID != MBM_INV_DESC) {
-    ::mbm_cancel_request(m_bmID);
-  }
+  if ( !m_cancelled && m_bmID != MBM_INV_DESC ) { ::mbm_cancel_request( m_bmID ); }
   m_cancelled = true;
 }
 
-void Allen::MBMOutput::output_done()
-{
+void Allen::MBMOutput::output_done() {
   m_cancelled = false;
-  if (m_bmID != MBM_INV_DESC) {
-    ::mbm_exclude(m_bmID);
-  }
+  if ( m_bmID != MBM_INV_DESC ) { ::mbm_exclude( m_bmID ); }
 }
diff --git a/AllenOnline/src/MBMOutput.h b/AllenOnline/src/MBMOutput.h
index c49985f84..a920d5249 100644
--- a/AllenOnline/src/MBMOutput.h
+++ b/AllenOnline/src/MBMOutput.h
@@ -21,62 +21,59 @@
 
 namespace Allen {
 
-class MBMOutput final : public Service, public OutputHandler {
-public:
-  MBMOutput(std::string name, ISvcLocator* loc);
+  class MBMOutput final : public Service, public OutputHandler {
+  public:
+    MBMOutput( std::string name, ISvcLocator* loc );
 
-  // std::unique_ptr<RTL::Logger>&& logger);
+    // std::unique_ptr<RTL::Logger>&& logger);
 
-  /// Callback when space is availible (not really used)
-  static int spaceCallback(void* /* param */);
+    /// Callback when space is availible (not really used)
+    static int spaceCallback( void* /* param */ );
 
-  StatusCode initialize() override;
+    StatusCode initialize() override;
 
-  StatusCode start() override;
+    StatusCode start() override;
 
-protected:
+  protected:
+    std::tuple<size_t, gsl::span<char>> buffer( size_t buffer_size, size_t n_events ) override;
 
-  std::tuple<size_t, gsl::span<char>> buffer(size_t buffer_size, size_t n_events) override;
+    bool write_buffer( size_t ) override;
 
-  bool write_buffer(size_t) override;
+    void output_done() override;
 
-  void output_done() override;
+    void cancel() override;
 
-  void cancel() override;
+  private:
+    SmartIF<AllenConfiguration> m_allenConfig;
 
-private:
+    // Output buffer name
+    Gaudi::Property<std::string> m_bufferName{this, "BufferName", "Output"};
 
-  SmartIF<AllenConfiguration> m_allenConfig;
+    // Do output checksum
+    Gaudi::Property<bool> m_checksum{this, "Checksum", false};
 
-  // Output buffer name
-  Gaudi::Property<std::string> m_bufferName{this, "BufferName", "Output"};
+    // Process name
+    std::string m_processName;
 
-  // Do output checksum
-  Gaudi::Property<bool> m_checksum {this, "Checksum", false};
+    // partition ID
+    unsigned m_partitionID = 0;
 
-  // Process name
-  std::string m_processName;
+    // MBM buffer
+    gsl::span<char> m_buffer;
 
-  // partition ID
-  unsigned m_partitionID = 0;
+    // Output buffer ID
+    BMID m_bmID = MBM_INV_DESC;
 
-  // MBM buffer
-  gsl::span<char> m_buffer;
+    bool m_cancelled = false;
 
-  // Output buffer ID
-  BMID m_bmID = MBM_INV_DESC;
+    std::mutex                         m_mutex;
+    size_t                             m_id = 0;
+    std::list<size_t>                  m_freeIDs;
+    std::unordered_map<size_t, size_t> m_eventCounts;
 
-  bool m_cancelled = false;
-
-  std::mutex m_mutex;
-  size_t m_id = 0;
-  std::list<size_t> m_freeIDs;
-  std::unordered_map<size_t, size_t> m_eventCounts;
-
-  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_burstsOutput;
-  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsOutput;
-  std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbOutput;
-
-};
+    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_burstsOutput;
+    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsOutput;
+    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbOutput;
+  };
 
 } // namespace Allen
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 5bc00ca6a..fded95fb3 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -2,37 +2,36 @@
 * (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
 \*****************************************************************************/
 
-#include <Allen/Logger.h>
+#include <Allen/BankMapping.h>
+#include <Allen/BankTypes.h>
 #include <Allen/InputProvider.h>
+#include <Allen/Logger.h>
 #include <Allen/Provider.h>
-#include <Allen/BankTypes.h>
-#include <Allen/BankMapping.h>
-#include <Allen/Timer.h>
 #include <Allen/SliceUtils.h>
-#include <Allen/write_mdf.hpp>
+#include <Allen/Timer.h>
 #include <Allen/sourceid.h>
+#include <Allen/write_mdf.hpp>
 #include <Backend/BackendCommon.h>
 
-#include <MDF/StreamDescriptor.h>
 #include <MDF/MDFHeader.h>
+#include <MDF/StreamDescriptor.h>
 
 #include <Event/RawBank.h>
 
 #include <MBM/Requirement.h>
 
-#include <AllenOnline/TransposeMEP.h>
 #include <AllenOnline/ReadMEP.h>
+#include <AllenOnline/TransposeMEP.h>
 
-#include "MEPProvider.h"
 #include "AllenConfiguration.h"
-
+#include "MEPProvider.h"
 
 #ifdef HAVE_MPI
-#include <AllenOnline/MPIConfig.h>
-#include <hwloc.h>
+#  include <AllenOnline/MPIConfig.h>
+#  include <hwloc.h>
 #endif
 
-DECLARE_COMPONENT(MEPProvider)
+DECLARE_COMPONENT( MEPProvider )
 
 // The MEPProvider has three possible sources: MPI, the BufferManager
 // and files.
@@ -52,113 +51,97 @@ DECLARE_COMPONENT(MEPProvider)
 // allocated by the BufferManager is registered with the device
 // runtime.
 
-MEPProvider::MEPProvider(std::string name, ISvcLocator* loc)
-  : Service {name, loc},
-    m_mfp_count {0}
-{
-}
+MEPProvider::MEPProvider( std::string name, ISvcLocator* loc ) : Service{name, loc}, m_mfp_count{0} {}
 
-EventIDs MEPProvider::event_ids(size_t slice_index, std::optional<size_t> first, std::optional<size_t> last) const
-{
+EventIDs MEPProvider::event_ids( size_t slice_index, std::optional<size_t> first, std::optional<size_t> last ) const {
   auto const& ids = m_event_ids[slice_index];
-  return {ids.begin() + (first ? *first : 0), ids.begin() + (last ? *last : ids.size())};
+  return {ids.begin() + ( first ? *first : 0 ), ids.begin() + ( last ? *last : ids.size() )};
 }
 
-BanksAndOffsets MEPProvider::banks(BankTypes bank_type, size_t slice_index) const
-{
-  auto ib = to_integral(bank_type);
-  assert(static_cast<size_t>(ib) < m_slices.size());
-  assert(slice_index < m_slices[ib].size());
+BanksAndOffsets MEPProvider::banks( BankTypes bank_type, size_t slice_index ) const {
+  auto ib = to_integral( bank_type );
+  assert( static_cast<size_t>( ib ) < m_slices.size() );
+  assert( slice_index < m_slices[ib].size() );
   auto const& slice = m_slices[ib][slice_index];
   // auto const& [banks, data_size, offsets, offsets_size] = m_slices[ib][slice_index];
 
   BanksAndOffsets bno;
-  auto& spans = bno.fragments;
-  spans.reserve(slice.fragments.size());
-  for (auto s : slice.fragments) {
-    spans.emplace_back(s);
-  }
+  auto&           spans = bno.fragments;
+  spans.reserve( slice.fragments.size() );
+  for ( auto s : slice.fragments ) { spans.emplace_back( s ); }
 
   bno.fragments_mem_size = m_transpose_mep.value() ? slice.offsets[slice.n_offsets - 1] : slice.fragments_mem_size;
-  bno.offsets = slice.offsets;
-  bno.sizes = slice.sizes;
-  bno.types = slice.types;
-  bno.version = m_banks_version[ib];
-
-  if (msgLevel(MSG::VERBOSE)) {
-    verbose() << "slice " << std::setw(3) << slice_index << " bank type " << ib
-              << " size " << std::setw(12) << bno.fragments_mem_size << endmsg;
+  bno.offsets            = slice.offsets;
+  bno.sizes              = slice.sizes;
+  bno.types              = slice.types;
+  bno.version            = m_banks_version[ib];
+
+  if ( msgLevel( MSG::VERBOSE ) ) {
+    verbose() << "slice " << std::setw( 3 ) << slice_index << " bank type " << ib << " size " << std::setw( 12 )
+              << bno.fragments_mem_size << endmsg;
   }
 
   return bno;
 }
 
-std::tuple<bool, bool, bool, size_t, size_t, uint>
-MEPProvider::get_slice(std::optional<unsigned int> timeout)
-{
-  bool timed_out = false, done = false;
-  size_t slice_index = 0, n_filled = 0;
-  uint run_no = 0;
-  std::unique_lock<std::mutex> lock {m_transpose_mut};
+std::tuple<bool, bool, bool, size_t, size_t, uint> MEPProvider::get_slice( std::optional<unsigned int> timeout ) {
+  bool                         timed_out = false, done = false;
+  size_t                       slice_index = 0, n_filled = 0;
+  uint                         run_no = 0;
+  std::unique_lock<std::mutex> lock{m_transpose_mut};
 
-  if (!m_read_error) {
+  if ( !m_read_error ) {
     // If no transposed slices are ready for processing, wait until
     // one is; use a timeout if requested
-    if (m_transposed.empty()) {
+    if ( m_transposed.empty() ) {
       auto wakeup = [this] {
         auto n_writable = count_writable();
-        return (
-          !m_transposed.empty() || m_read_error || (m_transpose_done && n_writable == m_buffer_status.size()) ||
-          (m_stopping && n_writable == m_buffer_status.size()));
+        return ( !m_transposed.empty() || m_read_error ||
+                 ( m_transpose_done && n_writable == m_buffer_status.size() ) ||
+                 ( m_stopping && n_writable == m_buffer_status.size() ) );
       };
-      if (timeout) {
-        timed_out = !m_transposed_cond.wait_for(lock, std::chrono::milliseconds {*timeout}, wakeup);
-      }
-      else {
-        m_transposed_cond.wait(lock, wakeup);
+      if ( timeout ) {
+        timed_out = !m_transposed_cond.wait_for( lock, std::chrono::milliseconds{*timeout}, wakeup );
+      } else {
+        m_transposed_cond.wait( lock, wakeup );
       }
     }
-    if (!m_read_error && !m_transposed.empty() && (!timeout || (timeout && !timed_out))) {
-      std::tie(slice_index, n_filled) = m_transposed.front();
+    if ( !m_read_error && !m_transposed.empty() && ( !timeout || ( timeout && !timed_out ) ) ) {
+      std::tie( slice_index, n_filled ) = m_transposed.front();
       m_transposed.pop_front();
-      if (n_filled > 0) {
-        run_no = std::get<0>(m_event_ids[slice_index].front());
-      }
+      if ( n_filled > 0 ) { run_no = std::get<0>( m_event_ids[slice_index].front() ); }
     }
   }
 
   // Check if I/O and transposition is done and return a slice index
   auto n_writable = count_writable();
-  done = ((m_transpose_done && m_transposed.empty()) || m_stopping) && n_writable == m_buffer_status.size();
+  done = ( ( m_transpose_done && m_transposed.empty() ) || m_stopping ) && n_writable == m_buffer_status.size();
 
-  if (timed_out && msgLevel(MSG::DEBUG)) {
-    debug() << "get_slice timed out; error " << m_read_error << " done " << done
-            << " n_filled " << n_filled << endmsg;
-  }
-  else if (!timed_out && msgLevel(MSG::DEBUG)) {
-    debug() << "get_slice returning " << slice_index << "; error " << m_read_error << " done "
-            << done << " n_filled " << n_filled << endmsg;
+  if ( timed_out && msgLevel( MSG::DEBUG ) ) {
+    debug() << "get_slice timed out; error " << m_read_error << " done " << done << " n_filled " << n_filled << endmsg;
+  } else if ( !timed_out && msgLevel( MSG::DEBUG ) ) {
+    debug() << "get_slice returning " << slice_index << "; error " << m_read_error << " done " << done << " n_filled "
+            << n_filled << endmsg;
   }
 
   return {!m_read_error, done, timed_out, slice_index, m_read_error ? 0 : n_filled, run_no};
 }
 
-void MEPProvider::slice_free(size_t slice_index)
-{
+void MEPProvider::slice_free( size_t slice_index ) {
   // Check if a slice was acually in use before and if it was, only
   // notify the transpose threads that a free slice is available
   bool freed = false, set_writable = false;
-  int i_buffer = 0;
+  int  i_buffer = 0;
   {
-    std::unique_lock<std::mutex> lock {m_slice_mut};
-    if (!m_slice_free[slice_index]) {
+    std::unique_lock<std::mutex> lock{m_slice_mut};
+    if ( !m_slice_free[slice_index] ) {
       m_slice_free[slice_index] = true;
-      freed = true;
+      freed                     = true;
 
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
 
       // Clear relation between slice and buffer
-      i_buffer = std::get<0>(m_slice_to_buffer[slice_index]);
+      i_buffer     = std::get<0>( m_slice_to_buffer[slice_index] );
       auto& status = m_buffer_status[i_buffer];
       --status.work_counter;
 
@@ -166,37 +149,35 @@ void MEPProvider::slice_free(size_t slice_index)
 
       // If MEPs are not transposed and the respective buffer is no
       // longer in use, set it to writable
-      if (
-        status.work_counter == 0 &&
-        (std::find_if(m_slice_to_buffer.begin(), m_slice_to_buffer.end(), [i_buffer](const auto& entry) {
-           return std::get<0>(entry) == i_buffer;
-         }) == m_slice_to_buffer.end())) {
+      if ( status.work_counter == 0 &&
+           ( std::find_if( m_slice_to_buffer.begin(), m_slice_to_buffer.end(), [i_buffer]( const auto& entry ) {
+               return std::get<0>( entry ) == i_buffer;
+             } ) == m_slice_to_buffer.end() ) ) {
         status.writable = true;
-        set_writable = true;
-        if (m_source == MEP::ProviderSource::MBM) {
-          std::unique_lock<std::mutex> lock {m_mbm_mutex};
-          if (m_buffer_event[i_buffer]) {
-            ::mbm_free_event(m_bmIDs[i_buffer]);
+        set_writable    = true;
+        if ( m_source == MEP::ProviderSource::MBM ) {
+          std::unique_lock<std::mutex> lock{m_mbm_mutex};
+          if ( m_buffer_event[i_buffer] ) {
+            ::mbm_free_event( m_bmIDs[i_buffer] );
             m_buffer_event[i_buffer] = false;
           }
         }
 
-        if (status.work_counter == 0) {
+        if ( status.work_counter == 0 ) {
           m_transpose_done =
-            m_done && std::all_of(m_buffer_status.begin(), m_buffer_status.end(), [](BufferStatus const& stat) {
-              return stat.intervals.empty() && stat.work_counter == 0;
-            });
-          if (m_transpose_done) {
-            m_transpose_cond.notify_all();
-          }
+              m_done && std::all_of( m_buffer_status.begin(), m_buffer_status.end(), []( BufferStatus const& stat ) {
+                return stat.intervals.empty() && stat.work_counter == 0;
+              } );
+          if ( m_transpose_done ) { m_transpose_cond.notify_all(); }
         }
 
-        if (msgLevel(MSG::DEBUG)) {
+        if ( msgLevel( MSG::DEBUG ) ) {
           debug() << "Freed MEP buffer " << i_buffer << "; writable: " << count_writable() << endmsg;
-          for (auto const& status :  m_buffer_status) {
-            verbose() << std::setw(4) << status.index << std::setw(3) << status.writable << std::setw(4) << status.work_counter;
-            for (auto interval : status.intervals) {
-              verbose() << std::setw(5) << std::get<0>(interval) << std::setw(5) << std::get<1>(interval);
+          for ( auto const& status : m_buffer_status ) {
+            verbose() << std::setw( 4 ) << status.index << std::setw( 3 ) << status.writable << std::setw( 4 )
+                      << status.work_counter;
+            for ( auto interval : status.intervals ) {
+              verbose() << std::setw( 5 ) << std::get<0>( interval ) << std::setw( 5 ) << std::get<1>( interval );
             }
             verbose() << endmsg;
           }
@@ -204,226 +185,206 @@ void MEPProvider::slice_free(size_t slice_index)
       }
     }
   }
-  if (freed) {
-    if (msgLevel(MSG::DEBUG)) debug() << "Freed slice " << slice_index << endmsg;
+  if ( freed ) {
+    if ( msgLevel( MSG::DEBUG ) ) debug() << "Freed slice " << slice_index << endmsg;
     m_slice_cond.notify_one();
   }
-  if (set_writable) {
-    if (msgLevel(MSG::DEBUG)) debug() << "Set buffer " << i_buffer << " writable" << endmsg;
+  if ( set_writable ) {
+    if ( msgLevel( MSG::DEBUG ) ) debug() << "Set buffer " << i_buffer << " writable" << endmsg;
     m_receive_cond.notify_all();
   }
 }
 
-void MEPProvider::event_sizes(
-  size_t const slice_index,
-  gsl::span<unsigned int const> const selected_events,
-  std::vector<size_t>& sizes) const
-{
-  int i_buffer = 0;
+void MEPProvider::event_sizes( size_t const slice_index, gsl::span<unsigned int const> const selected_events,
+                               std::vector<size_t>& sizes ) const {
+  int    i_buffer       = 0;
   size_t interval_start = 0, interval_end = 0;
-  std::tie(i_buffer, interval_start, interval_end) = m_slice_to_buffer[slice_index];
-  auto const& blocks = m_net_slices[i_buffer].blocks;
-  for (unsigned int i = 0; i < selected_events.size(); ++i) {
+  std::tie( i_buffer, interval_start, interval_end ) = m_slice_to_buffer[slice_index];
+  auto const& blocks                                 = m_net_slices[i_buffer].blocks;
+  for ( unsigned int i = 0; i < selected_events.size(); ++i ) {
     auto event = selected_events[i];
     sizes[i] +=
-      std::accumulate(blocks.begin(), blocks.end(), 0ul, [event, interval_start](size_t s, const auto& block) {
-        auto const fragment_size = block.bank_sizes[interval_start + event];
-        return s + bank_header_size + Allen::padded_bank_size(fragment_size);
-      });
+        std::accumulate( blocks.begin(), blocks.end(), 0ul, [event, interval_start]( size_t s, const auto& block ) {
+          auto const fragment_size = block.bank_sizes[interval_start + event];
+          return s + bank_header_size + Allen::padded_bank_size( fragment_size );
+        } );
   }
 }
 
-void MEPProvider::copy_banks(size_t const slice_index, unsigned int const event, gsl::span<char> buffer) const
-{
+void MEPProvider::copy_banks( size_t const slice_index, unsigned int const event, gsl::span<char> buffer ) const {
   auto [i_buffer, interval_start, interval_end] = m_slice_to_buffer[slice_index];
-  const auto mep_event = interval_start + event;
+  const auto mep_event                          = interval_start + event;
 
-  auto const& slice = m_net_slices[i_buffer];
-  size_t offset = 0;
+  auto const& slice  = m_net_slices[i_buffer];
+  size_t      offset = 0;
 
-  for (size_t i_block = 0; i_block < slice.blocks.size(); ++i_block) {
-    auto const& block = slice.blocks[i_block];
+  for ( size_t i_block = 0; i_block < slice.blocks.size(); ++i_block ) {
+    auto const& block  = slice.blocks[i_block];
     auto const* header = block.header;
 
     // All banks are taken directly from the block data to be able
     // to treat banks needed by Allen and banks not needed by Allen
     // in the same way
     auto const fragment_offset = slice.offsets[i_block][mep_event];
-    auto fragment_size = block.bank_sizes[mep_event];
-
-    assert((offset + bank_header_size + Allen::padded_bank_size(fragment_size)) <= static_cast<size_t>(buffer.size()));
-    offset += Allen::add_raw_bank(
-      block.bank_types[mep_event],
-      header->block_version,
-      header->src_id,
-      {block.payload + fragment_offset, fragment_size},
-      buffer.data() + offset);
+    auto       fragment_size   = block.bank_sizes[mep_event];
+
+    assert( ( offset + bank_header_size + Allen::padded_bank_size( fragment_size ) ) <=
+            static_cast<size_t>( buffer.size() ) );
+    offset += Allen::add_raw_bank( block.bank_types[mep_event], header->block_version, header->src_id,
+                                   {block.payload + fragment_offset, fragment_size}, buffer.data() + offset );
   }
 }
 
-StatusCode MEPProvider::initialize()
-{
-  m_slice_free.resize(m_nslices.value(), true);
-  m_event_ids.resize(m_nslices.value());
-  m_event_masks.resize(m_nslices.value());
+StatusCode MEPProvider::initialize() {
+  m_slice_free.resize( m_nslices.value(), true );
+  m_event_ids.resize( m_nslices.value() );
+  m_event_masks.resize( m_nslices.value() );
 
-  m_buffer_status.resize(n_buffers());
-  for (size_t i = 0; i <  m_buffer_status.size(); ++i) {
-    m_buffer_status[i].index = i;
-  }
+  m_buffer_status.resize( n_buffers() );
+  for ( size_t i = 0; i < m_buffer_status.size(); ++i ) { m_buffer_status[i].index = i; }
 
-  auto config = service("AllenConfiguration/AllenConfiguration", false).as<AllenConfiguration>();
-  if (!config) {
+  auto config = service( "AllenConfiguration/AllenConfiguration", false ).as<AllenConfiguration>();
+  if ( !config ) {
     error() << "Failed to retrieve AllenConfiguration." << endmsg;
     return StatusCode::FAILURE;
   }
   m_allenConfig = config.get();
 
-  m_bank_types = Allen::configured_bank_types(m_allenConfig->json.value());
-  if (m_bank_types.empty()) {
+  m_bank_types = Allen::configured_bank_types( m_allenConfig->json.value() );
+  if ( m_bank_types.empty() ) {
     error() << "No bank types specified" << endmsg;
     return StatusCode::FAILURE;
-  }
-  else {
+  } else {
     info() << "Providing banks for";
-    for (auto bt : m_bank_types) info() << " " << ::bank_name(bt);
+    for ( auto bt : m_bank_types ) info() << " " << ::bank_name( bt );
     info() << endmsg;
   }
 
-  auto eventsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Events", true).get());
-  if (!eventsSvc) {
+  auto eventsSvc = dynamic_cast<Service*>( service<IService>( "AllenIOMon/Events", true ).get() );
+  if ( !eventsSvc ) {
     error() << "Failed to obtain Events service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
 
-  auto burstsSvc = dynamic_cast<Service*>(service<IService>("AllenIOMon/Bursts", true).get());
-  if (!eventsSvc) {
+  auto burstsSvc = dynamic_cast<Service*>( service<IService>( "AllenIOMon/Bursts", true ).get() );
+  if ( !eventsSvc ) {
     error() << "Failed to obtain Bursts service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
 
-  m_mepsInput = std::make_unique<Gaudi::Accumulators::Counter<>>(burstsSvc, "IN");
-  m_eventsInput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "IN");
-  m_mbInput = std::make_unique<Gaudi::Accumulators::Counter<>>(eventsSvc, "MB_IN");
+  m_mepsInput   = std::make_unique<Gaudi::Accumulators::Counter<>>( burstsSvc, "IN" );
+  m_eventsInput = std::make_unique<Gaudi::Accumulators::Counter<>>( eventsSvc, "IN" );
+  m_mbInput     = std::make_unique<Gaudi::Accumulators::Counter<>>( eventsSvc, "MB_IN" );
 
-  #ifdef HAVE_MPI
-  if (!m_buffer_numa.value().empty() && m_buffer_numa.value().size() != n_buffers()) {
+#ifdef HAVE_MPI
+  if ( !m_buffer_numa.value().empty() && m_buffer_numa.value().size() != n_buffers() ) {
     error() << "Buffer NUMA domains must be specified for all buffers" << endmsg;
     return StatusCode::FAILURE;
   }
 
   // Allocate and initialize topology object.
-  hwloc_topology_init(&m_topology);
-
-  // discover everything, in particular I/O devices like
-  // InfiniBand cards
-  #if HWLOC_API_VERSION >= 0x20000
-  hwloc_topology_set_io_types_filter(m_topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT);
-  #else
-  hwloc_topology_set_flags(m_topology, HWLOC_TOPOLOGY_FLAG_WHOLE_SYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES);
-  #endif
+  hwloc_topology_init( &m_topology );
+
+// discover everything, in particular I/O devices like
+// InfiniBand cards
+#  if HWLOC_API_VERSION >= 0x20000
+  hwloc_topology_set_io_types_filter( m_topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT );
+#  else
+  hwloc_topology_set_flags( m_topology, HWLOC_TOPOLOGY_FLAG_WHOLE_SYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES );
+#  endif
   // Perform the topology detection.
-  hwloc_topology_load(m_topology);
+  hwloc_topology_load( m_topology );
 
-  auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
-  for (auto domain : m_buffer_numa) {
-    if (domain >= n_numa) {
+  auto n_numa = hwloc_get_nbobjs_by_type( m_topology, HWLOC_OBJ_NUMANODE );
+  for ( auto domain : m_buffer_numa ) {
+    if ( domain >= n_numa ) {
       error() << "Illegal NUMA domain specified: " << domain << endmsg;
       return StatusCode::FAILURE;
     }
   }
-  #endif
+#endif
 
   std::optional<size_t> n_events = std::nullopt;
-  if (m_nevents.value() >= 0)
-    n_events = static_cast<size_t>(m_nevents.value());
+  if ( m_nevents.value() >= 0 ) n_events = static_cast<size_t>( m_nevents.value() );
 
-  init_input(m_nslices, m_events_per_slice, m_bank_types,
-             m_transpose_mep.value() ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, n_events);
+  init_input( m_nslices, m_events_per_slice, m_bank_types,
+              m_transpose_mep.value() ? IInputProvider::Layout::Allen : IInputProvider::Layout::MEP, n_events );
 
-  if (m_transpose_mep) {
+  if ( m_transpose_mep ) {
     info() << "Providing events in Allen layout by transposing MEPs" << endmsg;
-  }
-  else {
-    info() << "Providing events in MEP layout" << endmsg;;
+  } else {
+    info() << "Providing events in MEP layout" << endmsg;
+    ;
   }
 
   m_buffer_reading = m_buffer_status.begin();
 
   StatusCode sc = StatusCode::SUCCESS;
-  if (m_source == MEP::ProviderSource::MPI) {
+  if ( m_source == MEP::ProviderSource::MPI ) {
     sc = init_mpi();
-    if (!sc.isSuccess()) return sc;
-  }
-  else if (m_source == MEP::ProviderSource::Files) {
-    m_read_buffers.resize(n_buffers());
+    if ( !sc.isSuccess() ) return sc;
+  } else if ( m_source == MEP::ProviderSource::Files ) {
+    m_read_buffers.resize( n_buffers() );
     // Initialize the current input filename
     m_current = m_connections.begin();
   }
-  m_net_slices.resize(n_buffers());
+  m_net_slices.resize( n_buffers() );
 
   // Allocate space to store event ids
-  for (size_t n = 0; n < m_nslices.value(); ++n) {
-    m_event_ids[n].reserve(m_events_per_slice.value());
-    m_event_masks[n].resize(m_events_per_slice.value(), 0);
+  for ( size_t n = 0; n < m_nslices.value(); ++n ) {
+    m_event_ids[n].reserve( m_events_per_slice.value() );
+    m_event_masks[n].resize( m_events_per_slice.value(), 0 );
   }
 
   return sc;
 }
 
-StatusCode MEPProvider::start()
-{
-  if (m_started) {
-    return StatusCode::SUCCESS;
-  }
+StatusCode MEPProvider::start() {
+  if ( m_started ) { return StatusCode::SUCCESS; }
 
-  std::unique_lock<std::mutex> lock {m_control_mutex};
+  std::unique_lock<std::mutex> lock{m_control_mutex};
   m_stopping = false;
-  m_done = false;
+  m_done     = false;
 
   // start MPI receive, MEP reading thread or BM thread
-  if (m_source == MEP::ProviderSource::MPI && m_input_threads.empty()) {
+  if ( m_source == MEP::ProviderSource::MPI && m_input_threads.empty() ) {
 #ifdef HAVE_MPI
     m_ninput_threads = 1;
-    m_input_threads.emplace_back(std::thread{&MEPProvider::mpi_read, this});
+    m_input_threads.emplace_back( std::thread{&MEPProvider::mpi_read, this} );
 #else
-    throw StrException {"MPI requested, but no MPI support built in."};
+    throw StrException{"MPI requested, but no MPI support built in."};
 #endif
-  }
-  else if (m_source == MEP::ProviderSource::Files && m_input_threads.empty()) {
+  } else if ( m_source == MEP::ProviderSource::Files && m_input_threads.empty() ) {
     m_ninput_threads = 1;
-    m_input_threads.emplace_back(std::thread{&MEPProvider::mep_read, this});
-  }
-  else if (m_source == MEP::ProviderSource::MBM) {
+    m_input_threads.emplace_back( std::thread{&MEPProvider::mep_read, this} );
+  } else if ( m_source == MEP::ProviderSource::MBM ) {
     auto sc = init_bm();
-    if (sc != MBM_NORMAL) {
-      return StatusCode::FAILURE;
-    }
+    if ( sc != MBM_NORMAL ) { return StatusCode::FAILURE; }
     m_ninput_threads = m_thread_per_buffer.value() ? m_connections.size() : 1;
-    if (m_thread_per_buffer.value()) {
-      for (auto buffer : m_connections.value()) {
+    if ( m_thread_per_buffer.value() ) {
+      for ( auto buffer : m_connections.value() ) {
         debug() << "Starting bm_read thread for " << buffer << endmsg;
-        m_input_threads.emplace_back(std::thread{&MEPProvider::bm_read, this, buffer});
+        m_input_threads.emplace_back( std::thread{&MEPProvider::bm_read, this, buffer} );
       }
     } else {
-      m_input_threads.emplace_back(std::thread{&MEPProvider::bm_read, this, ""});
+      m_input_threads.emplace_back( std::thread{&MEPProvider::bm_read, this, ""} );
     }
   }
 
   // Start the transpose threads
-  if (m_transpose_threads.empty() && !m_read_error) {
-    for (int i = 0; i < std::get<1>(m_bufferConfig.value()); ++i) {
-      m_transpose_threads.emplace_back([this, i] { transpose(i); });
+  if ( m_transpose_threads.empty() && !m_read_error ) {
+    for ( int i = 0; i < std::get<1>( m_bufferConfig.value() ); ++i ) {
+      m_transpose_threads.emplace_back( [this, i] { transpose( i ); } );
     }
   }
 
   bool const sizes_known = m_sizes_known;
-  m_start_cond.wait(lock, [this] { return m_input_started == m_input_threads.size(); });
+  m_start_cond.wait( lock, [this] { return m_input_started == m_input_threads.size(); } );
 
   debug() << "Input threads started" << endmsg;
 
   m_started = true;
-  if (!sizes_known) {
+  if ( !sizes_known ) {
     m_control_cond.notify_one();
   } else {
     m_control_cond.notify_all();
@@ -431,29 +392,26 @@ StatusCode MEPProvider::start()
   return StatusCode::SUCCESS;
 };
 
-StatusCode MEPProvider::stop()
-{
+StatusCode MEPProvider::stop() {
   {
-    std::unique_lock<std::mutex> lock {m_control_mutex};
+    std::unique_lock<std::mutex> lock{m_control_mutex};
     m_stopping = true;
   }
 
-  if (m_source == MEP::ProviderSource::MBM) {
+  if ( m_source == MEP::ProviderSource::MBM ) {
     {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
       debug() << "Cancelling MBM requests; n_writable: " << count_writable() << endmsg;
       // Cancel all requests to the buffer manager for those who are waiting
-      for (size_t b = 0; b < m_buffer_status.size(); ++b) {
-        if (!m_buffer_status[b].writable) ::mbm_cancel_request(m_bmIDs[b]);
+      for ( size_t b = 0; b < m_buffer_status.size(); ++b ) {
+        if ( !m_buffer_status[b].writable ) ::mbm_cancel_request( m_bmIDs[b] );
       }
     }
 
     // Notify input threads in case they're waiting and there hasn't been any data.
     m_control_cond.notify_all();
 
-    for (auto& input_thread : m_input_threads) {
-      input_thread.join();
-    }
+    for ( auto& input_thread : m_input_threads ) { input_thread.join(); }
     m_input_threads.clear();
 
     m_done = true;
@@ -462,8 +420,8 @@ StatusCode MEPProvider::stop()
   }
 
   {
-    std::unique_lock<std::mutex> lock {m_control_mutex};
-    m_started = false;
+    std::unique_lock<std::mutex> lock{m_control_mutex};
+    m_started       = false;
     m_input_started = 0;
   }
 
@@ -474,164 +432,141 @@ StatusCode MEPProvider::stop()
   return StatusCode::SUCCESS;
 };
 
-bool MEPProvider::release_buffers()
-{
-  for (size_t b = 0; b < m_registered_buffers.size(); ++b) {
+bool MEPProvider::release_buffers() {
+  for ( size_t b = 0; b < m_registered_buffers.size(); ++b ) {
     auto const* buffer_address = m_registered_buffers[b];
     try {
-      Allen::host_unregister(const_cast<char*>(buffer_address));
-      debug() << "Successfully uregistered BM memory for buffer " << b
-              << " with device runtime." << endmsg;
-    } catch (const std::invalid_argument& e) {
-      error() << "Failed to unregister BM memory for buffer " << b
-              << " with device runtime : " << e.what() << endmsg;
+      Allen::host_unregister( const_cast<char*>( buffer_address ) );
+      debug() << "Successfully uregistered BM memory for buffer " << b << " with device runtime." << endmsg;
+    } catch ( const std::invalid_argument& e ) {
+      error() << "Failed to unregister BM memory for buffer " << b << " with device runtime : " << e.what() << endmsg;
       return false;
     }
   }
   m_registered_buffers.clear();
 
-  for (size_t b = 0; b < m_bmIDs.size(); ++b) {
+  for ( size_t b = 0; b < m_bmIDs.size(); ++b ) {
     auto bmid = m_bmIDs[b];
-    if (bmid != MBM_INV_DESC) {
-      ::mbm_exclude(bmid);
-    }
+    if ( bmid != MBM_INV_DESC ) { ::mbm_exclude( bmid ); }
   }
   return true;
 }
 
-StatusCode MEPProvider::init_mpi()
-{
-  #ifdef HAVE_MPI
+StatusCode MEPProvider::init_mpi() {
+#ifdef HAVE_MPI
 
   // MPI initialization
-  auto len = name().length();
-  int provided = 0;
-  m_mpiArgv = new char*[1];
-  m_mpiArgv[0] = new char[len];
-  ::strncpy(m_mpiArgv[0], name().c_str(), len);
-  MPI_Init_thread(&m_mpiArgc, &m_mpiArgv, MPI_THREAD_MULTIPLE, &provided);
-  if (provided != MPI_THREAD_MULTIPLE) {
+  auto len      = name().length();
+  int  provided = 0;
+  m_mpiArgv     = new char*[1];
+  m_mpiArgv[0]  = new char[len];
+  ::strncpy( m_mpiArgv[0], name().c_str(), len );
+  MPI_Init_thread( &m_mpiArgc, &m_mpiArgv, MPI_THREAD_MULTIPLE, &provided );
+  if ( provided != MPI_THREAD_MULTIPLE ) {
     error() << "Failed to initialize MPI multi thread support." << endmsg;
     return StatusCode::FAILURE;
   }
 
   // Communication size
   int comm_size = 0;
-  MPI_Comm_size(MPI_COMM_WORLD, &comm_size);
-  if (comm_size > MPI::comm_size) {
+  MPI_Comm_size( MPI_COMM_WORLD, &comm_size );
+  if ( comm_size > MPI::comm_size ) {
     error() << "This program requires at most " << MPI::comm_size << " processes." << endmsg;
     return StatusCode::FAILURE;
   }
 
   // MPI: Who am I?
-  MPI_Comm_rank(MPI_COMM_WORLD, &m_rank);
+  MPI_Comm_rank( MPI_COMM_WORLD, &m_rank );
 
-  if (m_rank != MPI::receiver) {
+  if ( m_rank != MPI::receiver ) {
     error() << "AllenApplication can only function as MPI receiver." << endmsg;
     return StatusCode::FAILURE;
   }
 
-  m_domains.reserve(m_receivers.size());
+  m_domains.reserve( m_receivers.size() );
 
   hwloc_obj_t osdev = nullptr;
 
-  if (!m_receivers.empty()) {
+  if ( !m_receivers.empty() ) {
     // Find NUMA domain of receivers
-    while ((osdev = hwloc_get_next_osdev(m_topology, osdev))) {
+    while ( ( osdev = hwloc_get_next_osdev( m_topology, osdev ) ) ) {
       // We're interested in InfiniBand cards
-      if (osdev->attr->osdev.type == HWLOC_OBJ_OSDEV_OPENFABRICS) {
-        auto parent = hwloc_get_non_io_ancestor_obj(m_topology, osdev);
-        auto it = m_receivers.find(osdev->name);
-        if (it != m_receivers.end()) {
-          m_domains.emplace_back(it->second, parent->os_index);
-          debug() << "Located receiver device " << it->first
-                  << " in NUMA domain " << parent->os_index << endmsg;
+      if ( osdev->attr->osdev.type == HWLOC_OBJ_OSDEV_OPENFABRICS ) {
+        auto parent = hwloc_get_non_io_ancestor_obj( m_topology, osdev );
+        auto it     = m_receivers.find( osdev->name );
+        if ( it != m_receivers.end() ) {
+          m_domains.emplace_back( it->second, parent->os_index );
+          debug() << "Located receiver device " << it->first << " in NUMA domain " << parent->os_index << endmsg;
         }
       }
     }
-    if (m_domains.size() != m_receivers.size()) {
+    if ( m_domains.size() != m_receivers.size() ) {
       error() << "Failed to locate some receiver devices " << endmsg;
       return StatusCode::FAILURE;
     }
-  }
-  else {
+  } else {
     error() << "MPI requested, but no receivers specified" << endmsg;
     return StatusCode::FAILURE;
   }
 
   // Get last node. There's always at least one.
-  [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type(m_topology, HWLOC_OBJ_NUMANODE);
-  assert(static_cast<size_t>(n_numa) == m_domains.size());
-
-  std::vector<size_t> packing_factors(m_receivers.size());
-  for (size_t receiver = 0; receiver < m_receivers.size(); ++receiver) {
-    auto const receiver_rank = std::get<0>(m_domains[receiver]);
-    MPI_Recv(
-      &packing_factors[receiver],
-      1,
-      MPI_SIZE_T,
-      receiver_rank,
-      MPI::message::packing_factor,
-      MPI_COMM_WORLD,
-      MPI_STATUS_IGNORE);
+  [[maybe_unused]] auto n_numa = hwloc_get_nbobjs_by_type( m_topology, HWLOC_OBJ_NUMANODE );
+  assert( static_cast<size_t>( n_numa ) == m_domains.size() );
+
+  std::vector<size_t> packing_factors( m_receivers.size() );
+  for ( size_t receiver = 0; receiver < m_receivers.size(); ++receiver ) {
+    auto const receiver_rank = std::get<0>( m_domains[receiver] );
+    MPI_Recv( &packing_factors[receiver], 1, MPI_SIZE_T, receiver_rank, MPI::message::packing_factor, MPI_COMM_WORLD,
+              MPI_STATUS_IGNORE );
   }
 
-  if (!std::all_of(packing_factors.begin(), packing_factors.end(), [v = packing_factors.back()](auto const p) {
-        return p == v;
-      })) {
+  if ( !std::all_of( packing_factors.begin(), packing_factors.end(),
+                     [v = packing_factors.back()]( auto const p ) { return p == v; } ) ) {
     error() << "All MEPs must have the same packing factor" << endmsg;
     return StatusCode::FAILURE;
 
-  }
-  else {
+  } else {
     m_packing_factor = packing_factors.back();
   }
 
   // Allocate as many net slices as configured, of expected size
   // Packing factor can be done dynamically if needed
-  size_t n_bytes = std::lround(m_packing_factor * average_event_size * bank_size_fudge_factor * kB);
-  for (size_t i = 0; i < n_buffers(); ++i) {
+  size_t n_bytes = std::lround( m_packing_factor * average_event_size * bank_size_fudge_factor * kB );
+  for ( size_t i = 0; i < n_buffers(); ++i ) {
     char* contents = nullptr;
-    MPI_Alloc_mem(n_bytes, MPI_INFO_NULL, &contents);
+    MPI_Alloc_mem( n_bytes, MPI_INFO_NULL, &contents );
 
     // Only bind explicitly if there are multiple receivers,
     // otherwise assume a memory allocation policy is in effect
-    if (m_domains.size() > 1) {
-      auto numa_node = std::get<1>(m_domains[i % m_receivers.size()]);
-      auto sc = numa_membind(contents, n_bytes, numa_node);
-      if (sc.isFailure()) return sc;
+    if ( m_domains.size() > 1 ) {
+      auto numa_node = std::get<1>( m_domains[i % m_receivers.size()] );
+      auto sc        = numa_membind( contents, n_bytes, numa_node );
+      if ( sc.isFailure() ) return sc;
     }
 
-    Allen::host_register(contents, n_bytes, Allen::hostRegisterDefault);
-    m_net_slices[i] = {
-      nullptr,
-      {contents, n_bytes},
-      0u,
-      MEP::Blocks {},
-      MEP::SourceOffsets {},
-      n_bytes};
-    m_mpi_buffers.emplace_back(contents);
+    Allen::host_register( contents, n_bytes, Allen::hostRegisterDefault );
+    m_net_slices[i] = {nullptr, {contents, n_bytes}, 0u, MEP::Blocks{}, MEP::SourceOffsets{}, n_bytes};
+    m_mpi_buffers.emplace_back( contents );
   }
   return StatusCode::SUCCESS;
-  #else
+#else
   error() << "MPI requested, but Allen was not built with MPI support." << endmsg;
   return StatusCode::FAILURE;
-  #endif
+#endif
 }
 
-int MEPProvider::init_bm()
-{
-  m_bmIDs.resize(n_buffers());
+int MEPProvider::init_bm() {
+  m_bmIDs.resize( n_buffers() );
 
-  auto const partition = m_allenConfig->partition.value();
+  auto const partition   = m_allenConfig->partition.value();
   auto const partitionID = m_allenConfig->partitionID.value();
 
-  m_buffer_event.resize(n_buffers());
-  m_buffer_event.assign(n_buffers(), false);
+  m_buffer_event.resize( n_buffers() );
+  m_buffer_event.assign( n_buffers(), false );
 
-  std::vector<BMID> first(m_connections.size(), MBM_INV_DESC);
+  std::vector<BMID> first( m_connections.size(), MBM_INV_DESC );
 
-  for (size_t b = 0; b < n_buffers(); ++b) {
+  for ( size_t b = 0; b < n_buffers(); ++b ) {
     auto buffer_name = m_connections[b % m_connections.size()];
     if ( m_allenConfig->partitionBuffers.value() ) {
       std::stringstream stream;
@@ -640,11 +575,11 @@ int MEPProvider::init_bm()
       buffer_name += partition.empty() ? stream.str() : partition;
     }
 
-    auto pn = RTL::processName() + "." + std::to_string(b);
+    auto pn = RTL::processName() + "." + std::to_string( b );
     BMID bmid;
-    if (b < m_connections.size()) {
-      bmid = ::mbm_include_read(buffer_name.c_str(), pn.c_str(), partitionID, static_cast<int>(m_mbm_com.value()));
-      if (bmid == MBM_INV_DESC) {
+    if ( b < m_connections.size() ) {
+      bmid = ::mbm_include_read( buffer_name.c_str(), pn.c_str(), partitionID, static_cast<int>( m_mbm_com.value() ) );
+      if ( bmid == MBM_INV_DESC ) {
         error() << "MBM: Failed to connect to MBM buffer " << buffer_name << endmsg;
         return MBM_ERROR;
       }
@@ -653,33 +588,32 @@ int MEPProvider::init_bm()
       first[b % m_connections.size()] = bmid;
 
       // register buffer manager memory with the device runtime
-      size_t buffer_size = 0;
-      char const* buffer_address = mbm_buffer_address(bmid);
-      mbm_buffer_size(bmid, &buffer_size);
-      if (m_registered_buffers.size() < m_connections.size()) {
+      size_t      buffer_size    = 0;
+      char const* buffer_address = mbm_buffer_address( bmid );
+      mbm_buffer_size( bmid, &buffer_size );
+      if ( m_registered_buffers.size() < m_connections.size() ) {
         try {
-          Allen::host_register(const_cast<char*>(buffer_address), buffer_size, Allen::hostRegisterReadOnly);
-          debug() << "Successfully registered BM memory for buffer " << buffer_name
-                  << " with device runtime." << endmsg;
-        } catch (const std::invalid_argument& e) {
-          error() << "Failed to register BM memory for buffer " << buffer_name
-                  << " with device runtime : " << e.what() << endmsg;
+          Allen::host_register( const_cast<char*>( buffer_address ), buffer_size, Allen::hostRegisterReadOnly );
+          debug() << "Successfully registered BM memory for buffer " << buffer_name << " with device runtime."
+                  << endmsg;
+        } catch ( const std::invalid_argument& e ) {
+          error() << "Failed to register BM memory for buffer " << buffer_name << " with device runtime : " << e.what()
+                  << endmsg;
           return MBM_ERROR;
         }
-        m_registered_buffers.push_back(buffer_address);
+        m_registered_buffers.push_back( buffer_address );
       }
     } else {
-      bmid = ::mbm_connect(first[b % m_connections.size()], pn.c_str(), partitionID);
+      bmid = ::mbm_connect( first[b % m_connections.size()], pn.c_str(), partitionID );
     }
 
     using namespace std::chrono_literals;
-    std::this_thread::sleep_for(50ms);
+    std::this_thread::sleep_for( 50ms );
 
-    for(auto r : m_requests) {
+    for ( auto r : m_requests ) {
       MBM::Requirement rq{r};
-      int sc = ::mbm_add_req(bmid, rq.evtype, rq.trmask, rq.vetomask, rq.maskType,
-                             rq.userType, rq.freqType, rq.freq);
-      if (sc != MBM_NORMAL)   {
+      int sc = ::mbm_add_req( bmid, rq.evtype, rq.trmask, rq.vetomask, rq.maskType, rq.userType, rq.freqType, rq.freq );
+      if ( sc != MBM_NORMAL ) {
         error() << "MBM: Failed to add MBM requirement: " << r << endmsg;
         return MBM_ERROR;
       }
@@ -690,66 +624,60 @@ int MEPProvider::init_bm()
   return MBM_NORMAL;
 }
 
-size_t MEPProvider::count_writable() const
-{
-  return std::accumulate(m_buffer_status.begin(), m_buffer_status.end(), 0ul, [](size_t s, BufferStatus const& stat) {
-    return s + stat.writable;
-  });
+size_t MEPProvider::count_writable() const {
+  return std::accumulate( m_buffer_status.begin(), m_buffer_status.end(), 0ul,
+                          []( size_t s, BufferStatus const& stat ) { return s + stat.writable; } );
 }
 
-bool MEPProvider::allocate_storage(size_t i_read)
-{
-  if (m_sizes_known) return true;
+bool MEPProvider::allocate_storage( size_t i_read ) {
+  if ( m_sizes_known ) return true;
 
   // Count number of banks per flavour
   bool count_success = false;
 
   // Offsets are to the start of the event, which includes the header
-  auto& slice = m_net_slices[i_read];
-  auto const* mep = slice.mep;
-  size_t n_blocks = mep->header.n_MFPs;
+  auto&       slice    = m_net_slices[i_read];
+  auto const* mep      = slice.mep;
+  size_t      n_blocks = mep->header.n_MFPs;
 
   m_packing_factor = slice.packing_factor;
 
-  size_t const eps = m_events_per_slice.value();
-  auto n_interval = m_packing_factor / eps;
-  auto rest = m_packing_factor % eps;
-  for (auto& s : m_buffer_status) {
-    s.intervals.reserve(2 * (n_interval + rest));
-  }
+  size_t const eps        = m_events_per_slice.value();
+  auto         n_interval = m_packing_factor / eps;
+  auto         rest       = m_packing_factor % eps;
+  for ( auto& s : m_buffer_status ) { s.intervals.reserve( 2 * ( n_interval + rest ) ); }
 
-  for (auto& slice : m_net_slices) {
+  for ( auto& slice : m_net_slices ) {
     // The number of blocks in a MEP is known, use it to allocate
     // temporary storage used during transposition
-    slice.blocks.resize(n_blocks);
-    slice.offsets.resize(n_blocks);
-    for (auto& offsets : slice.offsets) {
-      offsets.resize(m_packing_factor + 1);
-      Allen::host_register(offsets.data(), offsets.capacity(), Allen::hostRegisterDefault);
+    slice.blocks.resize( n_blocks );
+    slice.offsets.resize( n_blocks );
+    for ( auto& offsets : slice.offsets ) {
+      offsets.resize( m_packing_factor + 1 );
+      Allen::host_register( offsets.data(), offsets.capacity(), Allen::hostRegisterDefault );
     }
   }
 
-  std::tie(count_success, m_mfp_count, m_banks_version) = MEP::fill_counts(mep);
+  std::tie( count_success, m_mfp_count, m_banks_version ) = MEP::fill_counts( mep );
 
   // Allocate slice memory that will contain transposed banks ready
   // for processing by the Allen kernels
-  auto size_fun = [this, eps](BankTypes bank_type) -> std::tuple<size_t, size_t, size_t> {
-    auto it = BankSizes.find(bank_type);
-    auto const sd_index = to_integral(bank_type);
+  auto size_fun = [this, eps]( BankTypes bank_type ) -> std::tuple<size_t, size_t, size_t> {
+    auto       it       = BankSizes.find( bank_type );
+    auto const sd_index = to_integral( bank_type );
 
-    auto aps = eps < 100 ? 100 : eps;
+    auto aps      = eps < 100 ? 100 : eps;
     auto n_blocks = m_mfp_count[sd_index];
-    auto n_sizes = aps * ((n_blocks + 2) / 2 + 1);
+    auto n_sizes  = aps * ( ( n_blocks + 2 ) / 2 + 1 );
 
-    if (it == end(BankSizes)) {
-      throw std::out_of_range {std::string {"Bank type "} + bank_name(bank_type) + " has no known size"};
+    if ( it == end( BankSizes ) ) {
+      throw std::out_of_range{std::string{"Bank type "} + bank_name( bank_type ) + " has no known size"};
     }
     // In case of direct MEP output, no memory should be allocated.
-    if (!m_transpose_mep.value()) {
+    if ( !m_transpose_mep.value() ) {
       // 0 to not allocate fragment memory; -1 to correct for +1 in allocate_slices: re-evaluate
-      return {0, n_sizes, 2 + n_blocks + (1 + eps) * (1 + n_blocks) - 2};
-    }
-    else {
+      return {0, n_sizes, 2 + n_blocks + ( 1 + eps ) * ( 1 + n_blocks ) - 2};
+    } else {
       // Lookup LHCb bank type corresponding to Allen bank type
 
       // When events are transposed from the read buffer into
@@ -757,52 +685,47 @@ bool MEPProvider::allocate_storage(size_t i_read)
       // to see if there is enough space available in a slice.
       // To avoid having to read every event twice to get the
       // size of all the banks.
-      auto n_bytes = std::lround(
-        ((1 + m_mfp_count[sd_index]) * sizeof(uint32_t) + it->second * kB) * aps *
-          bank_size_fudge_factor +
-        2 * MB); // FIXME for the banks_count
+      auto n_bytes = std::lround( ( ( 1 + m_mfp_count[sd_index] ) * sizeof( uint32_t ) + it->second * kB ) * aps *
+                                      bank_size_fudge_factor +
+                                  2 * MB ); // FIXME for the banks_count
       return {n_bytes, n_sizes, eps};
     }
   };
-  m_slices = allocate_slices(m_nslices, m_bank_types, size_fun);
+  m_slices = allocate_slices( m_nslices, m_bank_types, size_fun );
 
-  m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>(m_nslices, std::tuple{-1, 0ul, 0ul});
+  m_slice_to_buffer = std::vector<std::tuple<int, size_t, size_t>>( m_nslices, std::tuple{-1, 0ul, 0ul} );
 
-  if (!count_success) {
+  if ( !count_success ) {
     error() << "Failed to determine bank counts" << endmsg;
     return false;
-  }
-  else {
+  } else {
     m_sizes_known = true;
     return true;
   }
 }
 
-bool MEPProvider::open_file() const
-{
+bool MEPProvider::open_file() const {
   bool good = false;
 
   // Check if there are still files available
-  while (!good) {
+  while ( !good ) {
     // If looping on input is configured, do it
-    if (m_current == m_connections.end()) {
-      if (m_non_stop.value()) {
+    if ( m_current == m_connections.end() ) {
+      if ( m_non_stop.value() ) {
         m_current = m_connections.begin();
-      }
-      else {
+      } else {
         break;
       }
     }
 
-    if (m_input) m_input->close();
+    if ( m_input ) m_input->close();
 
-    m_input = LHCb::StreamDescriptor::bind(*m_current);
-    if (m_input->ioDesc > 0) {
+    m_input = LHCb::StreamDescriptor::bind( *m_current );
+    if ( m_input->ioDesc > 0 ) {
       info() << "Opened " << *m_current << endmsg;
       good = true;
-    }
-    else {
-      error() << "Failed to open " << *m_current << " " << strerror(errno) << endmsg;
+    } else {
+      error() << "Failed to open " << *m_current << " " << strerror( errno ) << endmsg;
       return false;
     }
     ++m_current;
@@ -810,80 +733,73 @@ bool MEPProvider::open_file() const
   return good;
 }
 
-std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t> MEPProvider::get_mep_buffer(
-  std::function<bool(IInputProvider::BufferStatus const&)> pred,
-  std::function<bool()> wait_pred,
-  std::vector<IInputProvider::BufferStatus>::iterator start,
-  std::condition_variable& cond,
-  std::unique_lock<std::mutex>& lock)
-{
+std::tuple<std::vector<IInputProvider::BufferStatus>::iterator, size_t>
+MEPProvider::get_mep_buffer( std::function<bool( IInputProvider::BufferStatus const& )> pred,
+                             std::function<bool()> wait_pred, std::vector<IInputProvider::BufferStatus>::iterator start,
+                             std::condition_variable& cond, std::unique_lock<std::mutex>& lock ) {
   // Obtain a prefetch buffer to read into, if none is available,
   // wait until one of the transpose threads is done with its
   // prefetch buffer
   auto find_buffer = [this, start, &pred] {
-    auto it = std::find_if(start, m_buffer_status.end(), pred);
-    if (it == m_buffer_status.end()) {
-      it = std::find_if(m_buffer_status.begin(), start, pred);
-      if (it == start) it = m_buffer_status.end();
+    auto it = std::find_if( start, m_buffer_status.end(), pred );
+    if ( it == m_buffer_status.end() ) {
+      it = std::find_if( m_buffer_status.begin(), start, pred );
+      if ( it == start ) it = m_buffer_status.end();
     }
     return it;
   };
 
   auto it = find_buffer();
-  if (it == m_buffer_status.end() && !m_transpose_done) {
-    cond.wait(lock, [this, &it, &find_buffer, wait_pred] {
+  if ( it == m_buffer_status.end() && !m_transpose_done ) {
+    cond.wait( lock, [this, &it, &find_buffer, wait_pred] {
       it = find_buffer();
-      return it != m_buffer_status.end() || m_read_error  || wait_pred();
-    });
+      return it != m_buffer_status.end() || m_read_error || wait_pred();
+    } );
   }
-  return {it, distance(m_buffer_status.begin(), it)};
+  return {it, distance( m_buffer_status.begin(), it )};
 }
 
-bool MEPProvider::prepare_mep(size_t i_buffer, size_t n_events)
-{
+bool MEPProvider::prepare_mep( size_t i_buffer, size_t n_events ) {
   auto& slice = m_net_slices[i_buffer];
 
-  if (slice.packing_factor > m_packing_factor) {
+  if ( slice.packing_factor > m_packing_factor ) {
     error() << "MEP encountered with packing factor larger than first MEP." << endmsg;
     read_error();
     return false;
   }
 
   // Fill blocks
-  MEP::find_blocks(slice.mep, slice.blocks);
+  MEP::find_blocks( slice.mep, slice.blocks );
 
   // Fill fragment offsets
-  MEP::fragment_offsets(slice.blocks, slice.offsets);
+  MEP::fragment_offsets( slice.blocks, slice.offsets );
 
-  if (n_events == 0) return true;
+  if ( n_events == 0 ) return true;
 
   auto& status = m_buffer_status[i_buffer];
-  assert(status.work_counter == 0);
+  assert( status.work_counter == 0 );
 
   auto& intervals = status.intervals;
 
-  size_t const eps = m_events_per_slice.value();
-  auto n_interval = n_events / eps;
-  auto rest = n_events % eps;
-  if (rest) {
-    if (msgLevel(MSG::DEBUG)) {
+  size_t const eps        = m_events_per_slice.value();
+  auto         n_interval = n_events / eps;
+  auto         rest       = n_events % eps;
+  if ( rest ) {
+    if ( msgLevel( MSG::DEBUG ) ) {
       debug() << "Set interval (rest): " << n_interval * eps << "," << n_interval * eps + rest << endmsg;
     }
-    intervals.emplace_back(n_interval * eps, n_interval * eps + rest);
+    intervals.emplace_back( n_interval * eps, n_interval * eps + rest );
   }
-  for (size_t i = n_interval; i != 0; --i) {
-    if (msgLevel(MSG::DEBUG)) {
-      debug() << "Set interval: " << (i - 1) * eps << "," << i * eps << endmsg;
-    }
-    intervals.emplace_back((i - 1) * eps, i * eps);
+  for ( size_t i = n_interval; i != 0; --i ) {
+    if ( msgLevel( MSG::DEBUG ) ) { debug() << "Set interval: " << ( i - 1 ) * eps << "," << i * eps << endmsg; }
+    intervals.emplace_back( ( i - 1 ) * eps, i * eps );
   }
   status.work_counter = intervals.size();
 
   return true;
 }
 
-void MEPProvider::read_error()
-{
+void MEPProvider::read_error() {
   m_read_error = true;
   m_transpose_cond.notify_all();
   m_receive_cond.notify_all();
@@ -891,85 +807,82 @@ void MEPProvider::read_error()
 }
 
 // mep reader thread
-void MEPProvider::mep_read()
-{
+void MEPProvider::mep_read() {
   bool receive_done = false;
 
-  size_t preloaded = 0;
-  std::vector<bool> preloaded_buffer(n_buffers(), false);
+  size_t            preloaded = 0;
+  std::vector<bool> preloaded_buffer( n_buffers(), false );
 
   auto to_read = this->n_events();
-  if (to_read && msgLevel(MSG::DEBUG)) debug() << "Reading " << *to_read << " events" << endmsg;
+  if ( to_read && msgLevel( MSG::DEBUG ) ) debug() << "Reading " << *to_read << " events" << endmsg;
   auto to_publish = 0;
 
-  while (!receive_done && !m_read_error) {
+  while ( !receive_done && !m_read_error ) {
     // If we've been stopped, wait for start or exit
-    if (!m_started) {
-      std::unique_lock<std::mutex> lock {m_control_mutex};
+    if ( !m_started ) {
+      std::unique_lock<std::mutex> lock{m_control_mutex};
       ++m_input_started;
-      if (m_input_started == m_ninput_threads) {
-        m_start_cond.notify_one();
-      }
+      if ( m_input_started == m_ninput_threads ) { m_start_cond.notify_one(); }
       debug() << "mep_read waiting for start" << endmsg;
-      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+      m_control_cond.wait( lock, [this] { return m_started || m_done; } );
     }
 
-    if (m_done) break;
+    if ( m_done ) break;
 
     // open the first file
-    if (!m_input && !open_file()) {
+    if ( !m_input && !open_file() ) {
       read_error();
       return;
     }
     size_t i_buffer;
     {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([](BufferStatus const& s) { return s.writable; },
-                       [this] { return m_stopping; },
-                       m_buffer_reading, m_receive_cond, lock);
-      if (m_buffer_reading != m_buffer_status.end()) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
+      std::tie( m_buffer_reading, i_buffer ) =
+          get_mep_buffer( []( BufferStatus const& s ) { return s.writable; }, [this] { return m_stopping; },
+                          m_buffer_reading, m_receive_cond, lock );
+      if ( m_buffer_reading != m_buffer_status.end() ) {
         m_buffer_reading->writable = false;
-        assert(m_buffer_reading->work_counter == 0);
-      }
-      else {
+        assert( m_buffer_reading->work_counter == 0 );
+      } else {
         continue;
       }
     }
-    if (m_done) {
+    if ( m_done ) {
       receive_done = true;
       break;
     }
 
-    if (msgLevel(MSG::DEBUG)) debug() << "Writing to MEP slice index " << i_buffer << endmsg;
+    if ( msgLevel( MSG::DEBUG ) ) debug() << "Writing to MEP slice index " << i_buffer << endmsg;
 
     auto& read_buffer = m_read_buffers[i_buffer];
-    auto& slice = m_net_slices[i_buffer];
+    auto& slice       = m_net_slices[i_buffer];
 
     bool success = false, eof = false;
 
-    while (!success || eof) {
-      if (!m_preload.value() || (preloaded < n_buffers() && !preloaded_buffer[i_buffer])) {
-        std::tie(eof, success, slice.mep, slice.packing_factor, slice.mep_data) = MEP::read_mep(*m_input, read_buffer);
-        if (!eof && msgLevel(MSG::DEBUG)) debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
+    while ( !success || eof ) {
+      if ( !m_preload.value() || ( preloaded < n_buffers() && !preloaded_buffer[i_buffer] ) ) {
+        std::tie( eof, success, slice.mep, slice.packing_factor, slice.mep_data ) =
+            MEP::read_mep( *m_input, read_buffer );
+        if ( !eof && msgLevel( MSG::DEBUG ) )
+          debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
 
-        #ifdef HAVE_MPI
-        if (success && !eof && m_preload.value() && !m_buffer_numa.value().empty()) {
+#ifdef HAVE_MPI
+        if ( success && !eof && m_preload.value() && !m_buffer_numa.value().empty() ) {
           auto const numa_node = m_buffer_numa[i_buffer];
-          auto const sc = numa_membind(read_buffer.data(), read_buffer.capacity(), numa_node);
-          if (sc.isSuccess()) {
+          auto const sc        = numa_membind( read_buffer.data(), read_buffer.capacity(), numa_node );
+          if ( sc.isSuccess() ) {
             info() << "Bound preloaded MEP read buffer " << i_buffer << " memory to node " << numa_node << endmsg;
-          }
-          else {
-            error() << "Failed to bind preloaded MEP read buffer " << i_buffer <<  " memory to node " << numa_node << " " << strerror(errno) << endmsg;
+          } else {
+            error() << "Failed to bind preloaded MEP read buffer " << i_buffer << " memory to node " << numa_node << " "
+                    << strerror( errno ) << endmsg;
             read_error();
             break;
           }
 
           // Register memory with CUDA
           try {
-            Allen::host_register(read_buffer.data(), read_buffer.capacity(), Allen::hostRegisterDefault);
-          } catch (std::invalid_argument const&) {
+            Allen::host_register( read_buffer.data(), read_buffer.capacity(), Allen::hostRegisterDefault );
+          } catch ( std::invalid_argument const& ) {
             read_error();
             break;
           }
@@ -977,84 +890,78 @@ void MEPProvider::mep_read()
           preloaded_buffer[i_buffer] = true;
           ++preloaded;
         }
-        #endif
-      }
-      else {
+#endif
+      } else {
         success = true;
-        eof = false;
-        if (msgLevel(MSG::DEBUG)) {
+        eof     = false;
+        if ( msgLevel( MSG::DEBUG ) ) {
           debug() << "Using MEP already read into buffer " << i_buffer << "; preloaded " << preloaded << endmsg;
         }
       }
 
-      if (!eof && success) {
-        if (to_read) {
-          to_publish = std::min(*to_read, size_t {slice.packing_factor});
+      if ( !eof && success ) {
+        if ( to_read ) {
+          to_publish = std::min( *to_read, size_t{slice.packing_factor} );
           *to_read -= to_publish;
-        }
-        else {
+        } else {
           to_publish = slice.packing_factor;
         }
       }
 
-      if ((success && to_read && *to_read == 0) || (eof && !open_file())) {
+      if ( ( success && to_read && *to_read == 0 ) || ( eof && !open_file() ) ) {
         // Try to open the next file, if there is none, prefetching
         // is done.
-        if (!m_read_error && msgLevel(MSG::DEBUG)) {
-          debug() << "Prefetch done" << endmsg;
-        }
+        if ( !m_read_error && msgLevel( MSG::DEBUG ) ) { debug() << "Prefetch done" << endmsg; }
         receive_done = true;
-        if (m_preload.value() && preloaded < n_buffers()) {
+        if ( m_preload.value() && preloaded < n_buffers() ) {
           error() << "Could not read sufficient MEPs for preloading" << endmsg;
           read_error();
         }
         break;
       }
 
-      if (!eof && !success) {
+      if ( !eof && !success ) {
         // Error encountered
         read_error();
         break;
       }
     }
 
-    if (m_read_error || (!m_sizes_known && !allocate_storage(i_buffer))) {
+    if ( m_read_error || ( !m_sizes_known && !allocate_storage( i_buffer ) ) ) {
       read_error();
       break;
     }
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
-    if (!m_read_error) {
+    if ( !m_read_error ) {
       {
-        std::unique_lock<std::mutex> lock {m_buffer_mutex};
+        std::unique_lock<std::mutex> lock{m_buffer_mutex};
 
         auto& status = m_buffer_status[i_buffer];
-        assert(status.work_counter == 0);
+        assert( status.work_counter == 0 );
 
-        auto const n_events = (to_read ? to_publish : size_t {slice.packing_factor});
+        auto const n_events = ( to_read ? to_publish : size_t{slice.packing_factor} );
 
-        if (!eof && to_publish != 0) {
+        if ( !eof && to_publish != 0 ) {
           // Monitor this MEP
-          (*m_mepsInput) += 1;
-          (*m_eventsInput) += n_events;
-          (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
+          ( *m_mepsInput ) += 1;
+          ( *m_eventsInput ) += n_events;
+          ( *m_mbInput ) += ( 2 * slice.mep->bytes() + 1 ) / ( 2 * 1024 * 1024 );
 
           // Set intervals for offset calculation/transposition
-          if (!prepare_mep(i_buffer, n_events)) break;
-        }
-        else {
+          if ( !prepare_mep( i_buffer, n_events ) ) break;
+        } else {
           // We didn't read anything, so free the buffer we got again
           status.writable = true;
         }
       }
-      if (receive_done) {
+      if ( receive_done ) {
         m_done = receive_done;
-        if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying all" << endmsg;
+        if ( msgLevel( MSG::DEBUG ) ) debug() << "Prefetch notifying all" << endmsg;
         m_transpose_cond.notify_all();
-      }
-      else if (!eof) {
-        if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
+      } else if ( !eof ) {
+        if ( msgLevel( MSG::DEBUG ) ) debug() << "Prefetch notifying one" << endmsg;
         m_transpose_cond.notify_one();
       }
     }
@@ -1062,94 +969,89 @@ void MEPProvider::mep_read()
 }
 
 // MPI reader thread
-void MEPProvider::mpi_read()
-{
-  #ifdef HAVE_MPI
+void MEPProvider::mpi_read() {
+#ifdef HAVE_MPI
 
-  std::vector<MPI_Request> requests(m_window_size);
+  std::vector<MPI_Request> requests( m_window_size );
 
   // Iterate over the slices
-  size_t reporting_period = 5;
-  std::vector<std::tuple<size_t, size_t>> data_received(m_receivers.size());
-  std::vector<size_t> n_meps(m_receivers.size());
-  Timer t;
-  Timer t_origin;
-  bool mpi_error = false;
-
-  for (size_t i = 0; i < m_receivers.size(); ++i) {
+  size_t                                  reporting_period = 5;
+  std::vector<std::tuple<size_t, size_t>> data_received( m_receivers.size() );
+  std::vector<size_t>                     n_meps( m_receivers.size() );
+  Timer                                   t;
+  Timer                                   t_origin;
+  bool                                    mpi_error = false;
+
+  for ( size_t i = 0; i < m_receivers.size(); ++i ) {
     auto [mpi_rank, numa_domain] = m_domains[i];
-    MPI_Recv(&n_meps[i], 1, MPI_SIZE_T, mpi_rank, MPI::message::number_of_meps, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
+    MPI_Recv( &n_meps[i], 1, MPI_SIZE_T, mpi_rank, MPI::message::number_of_meps, MPI_COMM_WORLD, MPI_STATUS_IGNORE );
   }
-  size_t number_of_meps = std::accumulate(n_meps.begin(), n_meps.end(), 0u);
+  size_t number_of_meps = std::accumulate( n_meps.begin(), n_meps.end(), 0u );
 
   size_t current_mep = 0;
-  while (!m_done && !m_read_error && (m_non_stop.value() || current_mep < number_of_meps)) {
+  while ( !m_done && !m_read_error && ( m_non_stop.value() || current_mep < number_of_meps ) ) {
     // If we've been stopped, wait for start or exit
-    if (!m_started) {
-      std::unique_lock<std::mutex> lock {m_control_mutex};
+    if ( !m_started ) {
+      std::unique_lock<std::mutex> lock{m_control_mutex};
       ++m_input_started;
-      if (m_input_started == m_ninput_threads) {
-        m_start_cond.notify_one();
-      }
+      if ( m_input_started == m_ninput_threads ) { m_start_cond.notify_one(); }
       debug() << "mpi_read waiting for start" << endmsg;
-      m_control_cond.wait(lock, [this] { return m_started || m_done; });
+      m_control_cond.wait( lock, [this] { return m_started || m_done; } );
     }
 
-    if (m_done) break;
+    if ( m_done ) break;
 
     // Obtain a prefetch buffer to read into, if none is available,
     // wait until one of the transpose threads is done with its
     // prefetch buffer
     size_t i_buffer;
     {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      std::tie(m_buffer_reading, i_buffer) =
-        get_mep_buffer([](BufferStatus const& s) { return s.writable; },
-                       [this] { return m_stopping; },
-                       m_buffer_reading, m_receive_cond, lock);
-      if (m_buffer_reading != m_buffer_status.end()) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
+      std::tie( m_buffer_reading, i_buffer ) =
+          get_mep_buffer( []( BufferStatus const& s ) { return s.writable; }, [this] { return m_stopping; },
+                          m_buffer_reading, m_receive_cond, lock );
+      if ( m_buffer_reading != m_buffer_status.end() ) {
         m_buffer_reading->writable = false;
-        assert(m_buffer_reading->work_counter == 0);
-      }
-      else {
+        assert( m_buffer_reading->work_counter == 0 );
+      } else {
         continue;
       }
     }
 
-    auto receiver = i_buffer % m_receivers.size();
+    auto receiver                 = i_buffer % m_receivers.size();
     auto [sender_rank, numa_node] = m_domains[receiver];
 
-    if (msgLevel(MSG::DEBUG)) {
-      debug() << "Receiving from rank " << sender_rank << " into buffer " << i_buffer
-              << " NUMA domain " << numa_node << endmsg;
+    if ( msgLevel( MSG::DEBUG ) ) {
+      debug() << "Receiving from rank " << sender_rank << " into buffer " << i_buffer << " NUMA domain " << numa_node
+              << endmsg;
     }
 
-    auto& slice = m_net_slices[i_buffer];
+    auto&  slice    = m_net_slices[i_buffer];
     char*& contents = m_mpi_buffers[i_buffer];
 
     size_t mep_size = 0;
-    MPI_Recv(&mep_size, 1, MPI_SIZE_T, sender_rank, MPI::message::event_size, MPI_COMM_WORLD, MPI_STATUS_IGNORE);
+    MPI_Recv( &mep_size, 1, MPI_SIZE_T, sender_rank, MPI::message::event_size, MPI_COMM_WORLD, MPI_STATUS_IGNORE );
 
     // Reallocate if needed
-    if (mep_size > slice.slice_size) {
+    if ( mep_size > slice.slice_size ) {
       slice.slice_size = mep_size * bank_size_fudge_factor;
       // Unregister memory
-      Allen::host_unregister(contents);
+      Allen::host_unregister( contents );
 
       // Free memory
-      MPI_Free_mem(contents);
+      MPI_Free_mem( contents );
 
       // Allocate new memory
-      MPI_Alloc_mem(slice.slice_size, MPI_INFO_NULL, &contents);
+      MPI_Alloc_mem( slice.slice_size, MPI_INFO_NULL, &contents );
 
       // Only bind explicitly if there are multiple receivers,
       // otherwise assume a memory allocation policy is in effect
-      if (m_domains.size() > 1) {
+      if ( m_domains.size() > 1 ) {
         // Bind memory to numa domain of receiving card
-        auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
-        auto s = hwloc_set_area_membind(
-          m_topology, contents, slice.slice_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET);
-        if (s != 0) {
+        auto numa_obj = hwloc_get_obj_by_type( m_topology, HWLOC_OBJ_NUMANODE, numa_node );
+        auto s = hwloc_set_area_membind( m_topology, contents, slice.slice_size, numa_obj->nodeset, HWLOC_MEMBIND_BIND,
+                                         HWLOC_MEMBIND_BYNODESET );
+        if ( s != 0 ) {
           read_error();
           break;
         }
@@ -1157,13 +1059,13 @@ void MEPProvider::mpi_read()
 
       // Register memory with CUDA
       try {
-        Allen::host_register(contents, slice.slice_size, Allen::hostRegisterDefault);
-      } catch (std::invalid_argument const&) {
+        Allen::host_register( contents, slice.slice_size, Allen::hostRegisterDefault );
+      } catch ( std::invalid_argument const& ) {
         read_error();
         break;
       }
 
-      slice.mep_data = gsl::span {contents, static_cast<events_size>(slice.slice_size)};
+      slice.mep_data = gsl::span{contents, static_cast<events_size>( slice.slice_size )};
     }
 
     // Number of full-size (MPI::mdf_chunk_size) messages
@@ -1174,56 +1076,38 @@ void MEPProvider::mpi_read()
     int n_sends = n_messages > m_window_size.value() ? m_window_size.value() : n_messages;
 
     // Initial parallel sends
-    for (int k = 0; k < n_sends; k++) {
+    for ( int k = 0; k < n_sends; k++ ) {
       char* message = contents + k * MPI::mdf_chunk_size;
-      MPI_Irecv(
-        message,
-        MPI::mdf_chunk_size,
-        MPI_BYTE,
-        sender_rank,
-        MPI::message::event_send_tag_start + k,
-        MPI_COMM_WORLD,
-        &requests[k]);
+      MPI_Irecv( message, MPI::mdf_chunk_size, MPI_BYTE, sender_rank, MPI::message::event_send_tag_start + k,
+                 MPI_COMM_WORLD, &requests[k] );
     }
     // Sliding window sends
-    for (int k = n_sends; k < n_messages; k++) {
+    for ( int k = n_sends; k < n_messages; k++ ) {
       int r;
-      MPI_Waitany(m_window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      MPI_Waitany( m_window_size, requests.data(), &r, MPI_STATUS_IGNORE );
       char* message = contents + k * MPI::mdf_chunk_size;
-      MPI_Irecv(
-        message,
-        MPI::mdf_chunk_size,
-        MPI_BYTE,
-        sender_rank,
-        MPI::message::event_send_tag_start + k,
-        MPI_COMM_WORLD,
-        &requests[r]);
+      MPI_Irecv( message, MPI::mdf_chunk_size, MPI_BYTE, sender_rank, MPI::message::event_send_tag_start + k,
+                 MPI_COMM_WORLD, &requests[r] );
     }
     // Last send (if necessary)
-    if (rest) {
+    if ( rest ) {
       int r;
-      MPI_Waitany(m_window_size, requests.data(), &r, MPI_STATUS_IGNORE);
+      MPI_Waitany( m_window_size, requests.data(), &r, MPI_STATUS_IGNORE );
       char* message = contents + n_messages * MPI::mdf_chunk_size;
-      MPI_Irecv(
-        message,
-        rest,
-        MPI_BYTE,
-        sender_rank,
-        MPI::message::event_send_tag_start + n_messages,
-        MPI_COMM_WORLD,
-        &requests[r]);
+      MPI_Irecv( message, rest, MPI_BYTE, sender_rank, MPI::message::event_send_tag_start + n_messages, MPI_COMM_WORLD,
+                 &requests[r] );
     }
     // Wait until all chunks have been sent
-    MPI_Waitall(n_sends, requests.data(), MPI_STATUSES_IGNORE);
+    MPI_Waitall( n_sends, requests.data(), MPI_STATUSES_IGNORE );
 
-    slice.mep = reinterpret_cast<EB::MEP const*>(contents);
-    slice.mep_data = gsl::span {contents, static_cast<events_size>(mep_size)};
+    slice.mep      = reinterpret_cast<EB::MEP const*>( contents );
+    slice.mep_data = gsl::span{contents, static_cast<events_size>( mep_size )};
 
-    auto const* mep = slice.mep;
-    auto const* mfp = mep->at(0);
+    auto const* mep      = slice.mep;
+    auto const* mfp      = mep->at( 0 );
     slice.packing_factor = mfp->header.n_banks;
 
-    if (!m_sizes_known && !allocate_storage(i_buffer)) {
+    if ( !m_sizes_known && !allocate_storage( i_buffer ) ) {
       read_error();
       break;
     }
@@ -1232,206 +1116,183 @@ void MEPProvider::mpi_read()
     bytes_received += mep_size;
     meps_received += 1;
 
-    (*m_mepsInput) += 1;
-    (*m_eventsInput) += slice.packing_factor;
-    (*m_mbInput) += (2 * mep_size + 1) / (2 * 1024 * 1024);
-
+    ( *m_mepsInput ) += 1;
+    ( *m_eventsInput ) += slice.packing_factor;
+    ( *m_mbInput ) += ( 2 * mep_size + 1 ) / ( 2 * 1024 * 1024 );
 
-    if (t.get_elapsed_time() >= reporting_period) {
-      const auto seconds = t.get_elapsed_time();
-      auto total_rate = 0.;
-      auto total_bandwidth = 0.;
-      for (size_t i_rec = 0; i_rec < m_receivers.size(); ++i_rec) {
-        auto& [mr, br] = data_received[i_rec];
+    if ( t.get_elapsed_time() >= reporting_period ) {
+      const auto seconds         = t.get_elapsed_time();
+      auto       total_rate      = 0.;
+      auto       total_bandwidth = 0.;
+      for ( size_t i_rec = 0; i_rec < m_receivers.size(); ++i_rec ) {
+        auto& [mr, br]            = data_received[i_rec];
         auto [rec_rank, rec_node] = m_domains[i_rec];
 
-        const double rate = (double) mr / seconds;
-        const double bandwidth = ((double) (br * 8)) / (1024 * 1024 * 1024 * seconds);
+        const double rate      = (double)mr / seconds;
+        const double bandwidth = ( (double)( br * 8 ) ) / ( 1024 * 1024 * 1024 * seconds );
         total_rate += rate;
         total_bandwidth += bandwidth;
-        printf(
-          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s; Domain %2i; Rank %2i\n",
-          t_origin.get_elapsed_time(),
-          seconds,
-          rate,
-          bandwidth,
-          rec_node,
-          rec_rank);
+        printf( "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s; Domain %2i; Rank %2i\n", t_origin.get_elapsed_time(),
+                seconds, rate, bandwidth, rec_node, rec_rank );
 
         br = 0;
         mr = 0;
       }
-      if (m_receivers.size() > 1) {
-        printf(
-          "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s\n",
-          t_origin.get_elapsed_time(),
-          seconds,
-          total_rate,
-          total_bandwidth);
+      if ( m_receivers.size() > 1 ) {
+        printf( "[%lf, %lf] Throughput: %lf MEP/s, %lf Gb/s\n", t_origin.get_elapsed_time(), seconds, total_rate,
+                total_bandwidth );
       }
       t.restart();
     }
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
-    if (!mpi_error) {
+    if ( !mpi_error ) {
       {
-        std::unique_lock<std::mutex> lock {m_buffer_mutex};
-        if (!prepare_mep(i_buffer, size_t {slice.packing_factor})) break;
+        std::unique_lock<std::mutex> lock{m_buffer_mutex};
+        if ( !prepare_mep( i_buffer, size_t{slice.packing_factor} ) ) break;
       }
-      if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying one" << endmsg;
+      if ( msgLevel( MSG::DEBUG ) ) debug() << "Prefetch notifying one" << endmsg;
       m_transpose_cond.notify_one();
     }
 
     current_mep++;
   }
 
-  if (!m_done) {
+  if ( !m_done ) {
     m_done = true;
-    if (msgLevel(MSG::DEBUG)) debug() << "Prefetch notifying all" << endmsg;
+    if ( msgLevel( MSG::DEBUG ) ) debug() << "Prefetch notifying all" << endmsg;
     m_transpose_cond.notify_all();
   }
-  #endif
+#endif
 }
 
-
 // buffer manager reader thread
-void MEPProvider::bm_read(const std::string& buffer_name)
-{
+void MEPProvider::bm_read( const std::string& buffer_name ) {
   auto const partitionID = m_allenConfig->partitionID.value();
-  auto to_read = this->n_events();
-  if (to_read)
+  auto       to_read     = this->n_events();
+  if ( to_read )
     error() << "Number of events makes no sense when receiving from"
             << " the buffer manager: ignoring" << endmsg;
 
-  size_t buffer_index = 0;
-  size_t const n_con = m_connections.size();
-  auto select_buffer = [&buffer_index, n_con](BufferStatus const& s) {
-    return s.writable && ((s.index % n_con) == buffer_index);
+  size_t       buffer_index  = 0;
+  size_t const n_con         = m_connections.size();
+  auto         select_buffer = [&buffer_index, n_con]( BufferStatus const& s ) {
+    return s.writable && ( ( s.index % n_con ) == buffer_index );
   };
 
   auto buffer_reading = m_buffer_status.begin();
-  if (m_thread_per_buffer.value()) {
-    auto it = std::find(m_connections.begin(), m_connections.end(), buffer_name);
-    buffer_index = std::distance(m_connections.begin(), it);
+  if ( m_thread_per_buffer.value() ) {
+    auto it        = std::find( m_connections.begin(), m_connections.end(), buffer_name );
+    buffer_index   = std::distance( m_connections.begin(), it );
     buffer_reading = m_buffer_status.begin() + buffer_index;
   }
 
-  while (!m_done && !m_stopping && !m_read_error) {
+  while ( !m_done && !m_stopping && !m_read_error ) {
     // If we've been stopped, wait for start or exit
-    if (!m_started) {
-      std::unique_lock<std::mutex> lock {m_control_mutex};
+    if ( !m_started ) {
+      std::unique_lock<std::mutex> lock{m_control_mutex};
       ++m_input_started;
       debug() << "bm_read " << buffer_name << " Waiting for start" << endmsg;
-      if (m_input_started == m_ninput_threads) {
-        m_start_cond.notify_one();
-      }
-      m_control_cond.wait(lock, [this] { return m_started || m_done || m_stopping; });
+      if ( m_input_started == m_ninput_threads ) { m_start_cond.notify_one(); }
+      m_control_cond.wait( lock, [this] { return m_started || m_done || m_stopping; } );
     }
 
-    if (m_done || m_stopping) break;
+    if ( m_done || m_stopping ) break;
 
     size_t i_buffer = 0;
     {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      std::tie(buffer_reading, i_buffer) =
-        get_mep_buffer(select_buffer,
-          [this] { return m_stopping; },
-          buffer_reading, m_receive_cond, lock);
-      if (buffer_reading != m_buffer_status.end()) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
+      std::tie( buffer_reading, i_buffer ) = get_mep_buffer(
+          select_buffer, [this] { return m_stopping; }, buffer_reading, m_receive_cond, lock );
+      if ( buffer_reading != m_buffer_status.end() ) {
         buffer_reading->writable = false;
-        assert(buffer_reading->work_counter == 0);
-      }
-      else {
+        assert( buffer_reading->work_counter == 0 );
+      } else {
         continue;
       }
     }
-    if (m_done) {
-      if (buffer_reading != m_buffer_status.end()) {
-        buffer_reading->writable = true;
-      }
+    if ( m_done ) {
+      if ( buffer_reading != m_buffer_status.end() ) { buffer_reading->writable = true; }
       break;
     }
 
-    if (msgLevel(MSG::DEBUG)) {
-      debug() << "Buffer " << buffer_name << " " << buffer_index << " writing to MEP slice index " << i_buffer << endmsg;
+    if ( msgLevel( MSG::DEBUG ) ) {
+      debug() << "Buffer " << buffer_name << " " << buffer_index << " writing to MEP slice index " << i_buffer
+              << endmsg;
     }
 
-    if (!m_thread_per_buffer.value()) {
-      buffer_index = (buffer_index + 1) % n_con;
-    }
+    if ( !m_thread_per_buffer.value() ) { buffer_index = ( buffer_index + 1 ) % n_con; }
 
     auto& slice = m_net_slices[i_buffer];
 
     bool cancelled = false;
 
     unsigned int trmask[BM_MASK_SIZE];
-    int ev_type = 0, *ev_data = 0;
-    long ev_len = 0;
+    int          ev_type = 0, *ev_data = 0;
+    long         ev_len = 0;
 
-    if (msgLevel(MSG::DEBUG)) debug() << "Waiting for MEP " << i_buffer << endmsg;
+    if ( msgLevel( MSG::DEBUG ) ) debug() << "Waiting for MEP " << i_buffer << endmsg;
 
-    #ifndef NDEBUG
+#ifndef NDEBUG
     {
-      std::unique_lock<std::mutex> lock {m_mbm_mutex};
-      assert(!m_buffer_event[i_buffer]);
+      std::unique_lock<std::mutex> lock{m_mbm_mutex};
+      assert( !m_buffer_event[i_buffer] );
     }
-    #endif
-    auto sc = ::mbm_get_event(m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, partitionID);
-    if (sc == MBM_NORMAL)  {
+#endif
+    auto sc = ::mbm_get_event( m_bmIDs[i_buffer], &ev_data, &ev_len, &ev_type, trmask, partitionID );
+    if ( sc == MBM_NORMAL ) {
       {
-        std::unique_lock<std::mutex> lock {m_mbm_mutex};
+        std::unique_lock<std::mutex> lock{m_mbm_mutex};
         m_buffer_event[i_buffer] = true;
       }
 
       // info() << "Got MEP " << i_buffer << endmsg;
-      slice.mep = reinterpret_cast<EB::MEP const*>(ev_data);
-      slice.mep_data = {reinterpret_cast<char const*>(ev_data), slice.mep->bytes()};
-      slice.slice_size = static_cast<size_t>(ev_len);
-      auto const* mfp = slice.mep->at(0);
+      slice.mep            = reinterpret_cast<EB::MEP const*>( ev_data );
+      slice.mep_data       = {reinterpret_cast<char const*>( ev_data ), slice.mep->bytes()};
+      slice.slice_size     = static_cast<size_t>( ev_len );
+      auto const* mfp      = slice.mep->at( 0 );
       slice.packing_factor = mfp->header.n_banks;
 
-      (*m_mepsInput) += 1;
-      (*m_eventsInput) += slice.packing_factor;
-      (*m_mbInput) += (2 * slice.mep->bytes() + 1) / (2 * 1024 * 1024);
+      ( *m_mepsInput ) += 1;
+      ( *m_eventsInput ) += slice.packing_factor;
+      ( *m_mbInput ) += ( 2 * slice.mep->bytes() + 1 ) / ( 2 * 1024 * 1024 );
 
-      if (msgLevel(MSG::DEBUG)) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
-    } else if (sc == MBM_REQ_CANCEL) {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+      if ( msgLevel( MSG::DEBUG ) ) debug() << "Got mep with packing factor " << slice.packing_factor << endmsg;
+    } else if ( sc == MBM_REQ_CANCEL ) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
       m_buffer_status[i_buffer].writable = true;
-      cancelled = true;
-      if (msgLevel(MSG::DEBUG)) debug() << "Got cancel" << endmsg;
+      cancelled                          = true;
+      if ( msgLevel( MSG::DEBUG ) ) debug() << "Got cancel" << endmsg;
     }
 
-    if (!m_sizes_known && !cancelled) {
-      if (!allocate_storage(i_buffer)) {
+    if ( !m_sizes_known && !cancelled ) {
+      if ( !allocate_storage( i_buffer ) ) {
         read_error();
         break;
-      }
-      else {
+      } else {
         m_control_cond.notify_all();
       }
     }
 
-    assert(cancelled || slice.packing_factor <= m_packing_factor);
+    assert( cancelled || slice.packing_factor <= m_packing_factor );
 
     // Notify a transpose thread that a new buffer of events is
     // ready. If prefetching is done, wake up all threads
-    if (!cancelled) {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      if (!prepare_mep(i_buffer, size_t {slice.packing_factor})) break;
+    if ( !cancelled ) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
+      if ( !prepare_mep( i_buffer, size_t{slice.packing_factor} ) ) break;
     }
 
-    if (!cancelled) {
+    if ( !cancelled ) {
       debug() << "Prefetch notifying one" << endmsg;
       m_transpose_cond.notify_one();
-    } else if (cancelled) {
+    } else if ( cancelled ) {
       break;
     }
   }
 }
 
-
 /**
  * @brief      Function to run in each thread transposing events
  *
@@ -1439,122 +1300,104 @@ void MEPProvider::bm_read(const std::string& buffer_name)
  *
  * @return     void
  */
-void MEPProvider::transpose(int thread_id)
-{
+void MEPProvider::transpose( int thread_id ) {
 
-  size_t i_buffer = 0;
+  size_t                     i_buffer = 0;
   std::tuple<size_t, size_t> interval;
-  std::optional<size_t> slice_index = std::nullopt;
+  std::optional<size_t>      slice_index = std::nullopt;
 
-  bool good = false, transpose_full = false;
+  bool   good = false, transpose_full = false;
   size_t n_transposed = 0;
 
-  auto has_intervals = [thread_id, n_input = m_ninput_threads](BufferStatus const& s) {
-    return !s.intervals.empty() && ((s.index % n_input) == (thread_id % n_input));
+  auto has_intervals = [thread_id, n_input = m_ninput_threads]( BufferStatus const& s ) {
+    return !s.intervals.empty() && ( ( s.index % n_input ) == ( thread_id % n_input ) );
   };
 
   std::vector<BufferStatus>::iterator buffer_transpose = m_buffer_status.begin();
 
-  while (!m_read_error && !m_transpose_done) {
+  while ( !m_read_error && !m_transpose_done ) {
     // Get a buffer to read from
     {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      std::tie(buffer_transpose, i_buffer) = get_mep_buffer(has_intervals,
-                                                            [this]() -> bool { return m_transpose_done; },
-                                                            buffer_transpose,
-                                                            m_transpose_cond, lock);
-      if (m_transpose_done || m_read_error) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
+      std::tie( buffer_transpose, i_buffer ) = get_mep_buffer(
+          has_intervals, [this]() -> bool { return m_transpose_done; }, buffer_transpose, m_transpose_cond, lock );
+      if ( m_transpose_done || m_read_error ) {
         break;
-      }
-      else if (buffer_transpose == m_buffer_status.end()) {
+      } else if ( buffer_transpose == m_buffer_status.end() ) {
         continue;
       }
       auto& status = *buffer_transpose;
-      assert(!status.intervals.empty());
+      assert( !status.intervals.empty() );
 
       interval = status.intervals.back();
       status.intervals.pop_back();
 
-      if (msgLevel(MSG::DEBUG)) {
-        debug() << "Transpose " << thread_id << ": Got MEP slice index " << i_buffer << " interval [" << std::get<0>(interval)
-                << "," << std::get<1>(interval) << ")" << endmsg;
+      if ( msgLevel( MSG::DEBUG ) ) {
+        debug() << "Transpose " << thread_id << ": Got MEP slice index " << i_buffer << " interval ["
+                << std::get<0>( interval ) << "," << std::get<1>( interval ) << ")" << endmsg;
       }
     }
 
     // Get a slice to write to
-    if (!slice_index) {
-      if (msgLevel(MSG::DEBUG)) debug() << "Transpose " << thread_id << ": Getting slice index" << endmsg;
+    if ( !slice_index ) {
+      if ( msgLevel( MSG::DEBUG ) ) debug() << "Transpose " << thread_id << ": Getting slice index" << endmsg;
       auto it = m_slice_free.end();
       {
-        std::unique_lock<std::mutex> lock {m_slice_mut};
-        it = find(m_slice_free.begin(), m_slice_free.end(), true);
-        if (it == m_slice_free.end()) {
-          if (msgLevel(MSG::DEBUG)) debug() << "Transpose " << thread_id << ": Waiting for free slice" << endmsg;
-          m_slice_cond.wait(lock, [this, &it] {
-            it = std::find(m_slice_free.begin(), m_slice_free.end(), true);
+        std::unique_lock<std::mutex> lock{m_slice_mut};
+        it = find( m_slice_free.begin(), m_slice_free.end(), true );
+        if ( it == m_slice_free.end() ) {
+          if ( msgLevel( MSG::DEBUG ) ) debug() << "Transpose " << thread_id << ": Waiting for free slice" << endmsg;
+          m_slice_cond.wait( lock, [this, &it] {
+            it = std::find( m_slice_free.begin(), m_slice_free.end(), true );
             return it != m_slice_free.end() || m_transpose_done;
-          });
+          } );
           // If transpose is done and there is no slice, we were
           // woken up be the desctructor before a slice was declared
           // free. In that case, exit without transposing
-          if (m_transpose_done && it == m_slice_free.end()) {
-            break;
-          }
+          if ( m_transpose_done && it == m_slice_free.end() ) { break; }
         }
-        *it = false;
-        slice_index = distance(m_slice_free.begin(), it);
-        if (msgLevel(MSG::DEBUG)) debug() << "Transpose " << thread_id << ": Got slice index " << *slice_index << endmsg;
+        *it         = false;
+        slice_index = distance( m_slice_free.begin(), it );
+        if ( msgLevel( MSG::DEBUG ) )
+          debug() << "Transpose " << thread_id << ": Got slice index " << *slice_index << endmsg;
       }
       {
-        std::unique_lock<std::mutex> lock {m_buffer_mutex};
+        std::unique_lock<std::mutex> lock{m_buffer_mutex};
         // Keep track of what buffer this slice belonged to
-        m_slice_to_buffer[*slice_index] = {i_buffer, std::get<0>(interval), std::get<1>(interval)};
+        m_slice_to_buffer[*slice_index] = {i_buffer, std::get<0>( interval ), std::get<1>( interval )};
       }
     }
 
     // Reset the slice
-    auto& event_ids = m_event_ids[*slice_index];
+    auto& event_ids  = m_event_ids[*slice_index];
     auto& event_mask = m_event_masks[*slice_index];
-    reset_slice(m_slices, *slice_index, m_bank_types, event_ids, !m_transpose_mep.value());
+    reset_slice( m_slices, *slice_index, m_bank_types, event_ids, !m_transpose_mep.value() );
 
     // MEP data
     auto& slice = m_net_slices[i_buffer];
 
     // Transpose or calculate offsets
-    if (m_transpose_mep.value()) {
+    if ( m_transpose_mep.value() ) {
       // Transpose the events into the slice
-      std::tie(good, transpose_full, n_transposed) = MEP::transpose_events(
-        m_slices,
-        *slice_index,
-        m_bank_types,
-        m_mfp_count,
-        event_ids,
-        event_mask,
-        slice.mep,
-        slice.blocks,
-        slice.offsets,
-        interval,
-        m_split_by_run.value());
-      if (msgLevel(MSG::DEBUG)) {
-        debug() << "Transpose " << thread_id << ": Transposed slice " << *slice_index
-                << "; good: " << good << "; full: " << transpose_full
-                << "; n_transposed: " << n_transposed << endmsg;
+      std::tie( good, transpose_full, n_transposed ) =
+          MEP::transpose_events( m_slices, *slice_index, m_bank_types, m_mfp_count, event_ids, event_mask, slice.mep,
+                                 slice.blocks, slice.offsets, interval, m_split_by_run.value() );
+      if ( msgLevel( MSG::DEBUG ) ) {
+        debug() << "Transpose " << thread_id << ": Transposed slice " << *slice_index << "; good: " << good
+                << "; full: " << transpose_full << "; n_transposed: " << n_transposed << endmsg;
       }
-    }
-    else {
+    } else {
       // Calculate fragment offsets in MEP per sub-detector
-      std::tie(good, transpose_full, n_transposed) = MEP::mep_offsets(
-        m_slices, *slice_index, m_bank_types, m_mfp_count, event_ids, event_mask, slice.blocks, interval);
-      if (msgLevel(MSG::DEBUG)) {
+      std::tie( good, transpose_full, n_transposed ) = MEP::mep_offsets(
+          m_slices, *slice_index, m_bank_types, m_mfp_count, event_ids, event_mask, slice.blocks, interval );
+      if ( msgLevel( MSG::DEBUG ) ) {
         debug() << "Transpose " << thread_id << ": Calculated MEP offsets for slice " << *slice_index
-                << "; good: " << good << "; full: " << transpose_full
-                << "; n_transposed: " << n_transposed << endmsg;
-
+                << "; good: " << good << "; full: " << transpose_full << "; n_transposed: " << n_transposed << endmsg;
       }
     }
 
-    if (m_read_error || !good) {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
+    if ( m_read_error || !good ) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
       m_read_error = true;
       m_transpose_cond.notify_one();
       break;
@@ -1562,36 +1405,32 @@ void MEPProvider::transpose(int thread_id)
 
     // Notify any threads waiting in get_slice that a slice is available
     {
-      std::unique_lock<std::mutex> lock {m_transpose_mut};
-      m_transposed.emplace_back(*slice_index, n_transposed);
+      std::unique_lock<std::mutex> lock{m_transpose_mut};
+      m_transposed.emplace_back( *slice_index, n_transposed );
     }
     m_transposed_cond.notify_one();
     slice_index.reset();
 
-    if (n_transposed != std::get<1>(interval) - std::get<0>(interval)) {
-      std::unique_lock<std::mutex> lock {m_buffer_mutex};
-      auto& status = m_buffer_status[i_buffer];
-        status.intervals.emplace_back(std::get<0>(interval) + n_transposed, std::get<1>(interval));
+    if ( n_transposed != std::get<1>( interval ) - std::get<0>( interval ) ) {
+      std::unique_lock<std::mutex> lock{m_buffer_mutex};
+      auto&                        status = m_buffer_status[i_buffer];
+      status.intervals.emplace_back( std::get<0>( interval ) + n_transposed, std::get<1>( interval ) );
     }
   }
 }
 
 #ifdef HAVE_MPI
-StatusCode MEPProvider::numa_membind(char const* mem, size_t size, int const numa_node) const {
-  auto numa_obj = hwloc_get_obj_by_type(m_topology, HWLOC_OBJ_NUMANODE, numa_node);
-  auto s = hwloc_set_area_membind(
-    m_topology, mem, size, numa_obj->nodeset, HWLOC_MEMBIND_BIND, HWLOC_MEMBIND_BYNODESET | HWLOC_MEMBIND_STRICT | HWLOC_MEMBIND_MIGRATE);
-  if (s != 0) {
-    error() << "Failed to bind memory to node " << numa_obj->os_index << " "
-            << strerror(errno) << endmsg;
+StatusCode MEPProvider::numa_membind( char const* mem, size_t size, int const numa_node ) const {
+  auto numa_obj = hwloc_get_obj_by_type( m_topology, HWLOC_OBJ_NUMANODE, numa_node );
+  auto s        = hwloc_set_area_membind( m_topology, mem, size, numa_obj->nodeset, HWLOC_MEMBIND_BIND,
+                                   HWLOC_MEMBIND_BYNODESET | HWLOC_MEMBIND_STRICT | HWLOC_MEMBIND_MIGRATE );
+  if ( s != 0 ) {
+    error() << "Failed to bind memory to node " << numa_obj->os_index << " " << strerror( errno ) << endmsg;
     return StatusCode::FAILURE;
-  }
-  else {
+  } else {
     return StatusCode::SUCCESS;
   }
 }
 #else
-StatusCode MEPProvider::numa_membind(char const*, size_t, int const) const {
-  return StatusCode::FAILURE;
-}
+StatusCode MEPProvider::numa_membind( char const*, size_t, int const ) const { return StatusCode::FAILURE; }
 #endif
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index 8b038b992..f27d62910 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -3,25 +3,25 @@
 \*****************************************************************************/
 #pragma once
 
-#include <thread>
-#include <vector>
+#include <algorithm>
 #include <array>
-#include <deque>
-#include <mutex>
 #include <atomic>
+#include <cassert>
 #include <chrono>
-#include <algorithm>
-#include <numeric>
 #include <condition_variable>
-#include <cassert>
+#include <deque>
+#include <mutex>
+#include <numeric>
+#include <thread>
+#include <vector>
 
-#include <unistd.h>
-#include <sys/types.h>
-#include <sys/stat.h>
 #include <fcntl.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+#include <unistd.h>
 
-#include <Gaudi/Property.h>
 #include <Gaudi/Accumulators.h>
+#include <Gaudi/Property.h>
 #include <Kernel/meta_enum.h>
 
 #include <GaudiKernel/Service.h>
@@ -36,31 +36,20 @@
 #include <AllenOnline/TransposeMEP.h>
 
 #ifdef HAVE_MPI
-#include <hwloc.h>
+#  include <hwloc.h>
 #endif
 
 namespace {
   using namespace Allen::Units;
   using namespace std::string_literals;
 
-  constexpr auto bank_header_size = sizeof(LHCb::RawBank) - sizeof(unsigned int);
+  constexpr auto bank_header_size = sizeof( LHCb::RawBank ) - sizeof( unsigned int );
 } // namespace
 
 namespace MEP {
-  meta_enum_class(ProviderSource, int,
-                  Unknown = 0,
-                  Files,
-                  MBM,
-                  MPI)
-
-  meta_enum_class(MBMCom, int,
-                  Unknown = 10,
-                  None = 0,
-                  FIFO = 1,
-                  Asio = 2,
-                  Unix = 4,
-                  SHM1 = 8,
-                  SHM2 = 9)
+  meta_enum_class( ProviderSource, int, Unknown = 0, Files, MBM, MPI )
+
+      meta_enum_class( MBMCom, int, Unknown = 10, None = 0, FIFO = 1, Asio = 2, Unix = 4, SHM1 = 8, SHM2 = 9 )
 }
 
 class AllenConfiguration;
@@ -93,7 +82,7 @@ class AllenConfiguration;
  */
 class MEPProvider final : public Service, public InputProvider {
 public:
-  MEPProvider(std::string name, ISvcLocator* loc);
+  MEPProvider( std::string name, ISvcLocator* loc );
 
   /**
    * @brief      Obtain event IDs of events stored in a given slice
@@ -102,8 +91,8 @@ public:
    *
    * @return     EventIDs of events in given slice
    */
-  EventIDs event_ids(size_t slice_index, std::optional<size_t> first = std::nullopt, std::optional<size_t> last = std::nullopt)
-    const override;
+  EventIDs event_ids( size_t slice_index, std::optional<size_t> first = std::nullopt,
+                      std::optional<size_t> last = std::nullopt ) const override;
 
   /**
    * @brief      Obtain banks from a slice
@@ -113,7 +102,7 @@ public:
    *
    * @return     Banks and their offsets
    */
-  BanksAndOffsets banks(BankTypes bank_type, size_t slice_index) const override;
+  BanksAndOffsets banks( BankTypes bank_type, size_t slice_index ) const override;
 
   /**
    * @brief      Get a slice that is ready for processing; thread-safe
@@ -122,8 +111,8 @@ public:
    *
    * @return     (good slice, timed out, slice index, number of events in slice)
    */
-  std::tuple<bool, bool, bool, size_t, size_t, uint> get_slice(
-    std::optional<unsigned int> timeout = std::nullopt) override;
+  std::tuple<bool, bool, bool, size_t, size_t, uint>
+  get_slice( std::optional<unsigned int> timeout = std::nullopt ) override;
 
   /**
    * @brief      Declare a slice free for reuse; thread-safe
@@ -132,33 +121,29 @@ public:
    *
    * @return     void
    */
-  void slice_free(size_t slice_index) override;
+  void slice_free( size_t slice_index ) override;
 
   bool release_buffers() override;
 
-  void event_sizes(
-    size_t const slice_index,
-    gsl::span<unsigned int const> const selected_events,
-    std::vector<size_t>& sizes) const override;
+  void event_sizes( size_t const slice_index, gsl::span<unsigned int const> const selected_events,
+                    std::vector<size_t>& sizes ) const override;
 
-  void copy_banks(size_t const slice_index, unsigned int const event, gsl::span<char> buffer) const override;
+  void copy_banks( size_t const slice_index, unsigned int const event, gsl::span<char> buffer ) const override;
 
   StatusCode initialize() override;
 
-
   StatusCode start() override;
 
   StatusCode stop() override;
 
 private:
-
   StatusCode init_mpi();
 
   int init_bm();
 
   size_t count_writable() const;
 
-  bool allocate_storage(size_t i_read);
+  bool allocate_storage( size_t i_read );
 
   /**
    * @brief      Open an input file; called from the prefetch thread
@@ -167,14 +152,12 @@ private:
    */
   bool open_file() const;
 
-  std::tuple<std::vector<BufferStatus>::iterator, size_t> get_mep_buffer(
-    std::function<bool(BufferStatus const&)> pred,
-    std::function<bool()> wait_pred,
-    std::vector<BufferStatus>::iterator start,
-    std::condition_variable& cond,
-    std::unique_lock<std::mutex>& lock);
+  std::tuple<std::vector<BufferStatus>::iterator, size_t>
+  get_mep_buffer( std::function<bool( BufferStatus const& )> pred, std::function<bool()> wait_pred,
+                  std::vector<BufferStatus>::iterator start, std::condition_variable& cond,
+                  std::unique_lock<std::mutex>& lock );
 
-  bool prepare_mep(size_t i_buffer, size_t n_events);
+  bool prepare_mep( size_t i_buffer, size_t n_events );
 
   void read_error();
 
@@ -185,7 +168,7 @@ private:
   void mpi_read();
 
   // buffer manager reader thread
-  void bm_read(const std::string& buffer_name);
+  void bm_read( const std::string& buffer_name );
 
   /**
    * @brief      Function to run in each thread transposing events
@@ -194,77 +177,77 @@ private:
    *
    * @return     void
    */
-  void transpose(int thread_id);
+  void transpose( int thread_id );
 
-  StatusCode numa_membind(char const* mem, size_t size, int const numa_node) const;
+  StatusCode numa_membind( char const* mem, size_t size, int const numa_node ) const;
 
   // Slices
-  size_t m_packing_factor = 0;
+  size_t                         m_packing_factor = 0;
   std::vector<std::vector<char>> m_read_buffers;
-  std::vector<char*> m_mpi_buffers;
-  MEP::Slices m_net_slices;
+  std::vector<char*>             m_mpi_buffers;
+  MEP::Slices                    m_net_slices;
 
   // data members for mpi thread
-  bool m_started = false;
-  bool m_stopping = false;
-  std::mutex m_control_mutex;
+  bool                    m_started  = false;
+  bool                    m_stopping = false;
+  std::mutex              m_control_mutex;
   std::condition_variable m_control_cond;
   std::condition_variable m_start_cond;
 
   // data members for mpi thread
-  std::mutex m_buffer_mutex;
+  std::mutex              m_buffer_mutex;
   std::condition_variable m_transpose_cond;
   std::condition_variable m_receive_cond;
 
-  #ifdef HAVE_MPI
+#ifdef HAVE_MPI
   std::vector<std::tuple<int, int>> m_domains;
-  hwloc_topology_t m_topology;
+  hwloc_topology_t                  m_topology;
 
   char** m_mpiArgv = nullptr;
-  int m_mpiArgc = 1;
-  int m_rank = -1;
-  #endif
+  int    m_mpiArgc = 1;
+  int    m_rank    = -1;
+#endif
 
-  std::vector<BufferStatus> m_buffer_status;
+  std::vector<BufferStatus>           m_buffer_status;
   std::vector<BufferStatus>::iterator m_buffer_reading;
 
-  std::mutex m_mbm_mutex;
-  std::vector<bool> m_buffer_event;
+  std::mutex               m_mbm_mutex;
+  std::vector<bool>        m_buffer_event;
   std::vector<std::thread> m_input_threads;
-  size_t m_ninput_threads = 0;
-  size_t m_input_started = 0;
+  size_t                   m_ninput_threads = 0;
+  size_t                   m_input_started  = 0;
 
   // Atomics to flag errors and completion
-  std::atomic<bool> m_done = false;
-  mutable std::atomic<bool> m_read_error = false;
-  std::atomic<bool> m_transpose_done = false;
+  std::atomic<bool>         m_done           = false;
+  mutable std::atomic<bool> m_read_error     = false;
+  std::atomic<bool>         m_transpose_done = false;
 
   // Memory slices, N for each raw bank type
-  Allen::Slices m_slices;
+  Allen::Slices                                m_slices;
   std::vector<std::tuple<int, size_t, size_t>> m_slice_to_buffer;
 
   // Array to store the version of banks per bank type
   mutable std::array<int, NBankTypes> m_banks_version;
 
   // Mutex, condition varaible and queue for parallel transposition of slices
-  std::mutex m_transpose_mut;
-  std::condition_variable m_transposed_cond;
+  std::mutex                             m_transpose_mut;
+  std::condition_variable                m_transposed_cond;
   std::deque<std::tuple<size_t, size_t>> m_transposed;
 
   // Keep track of what slices are free
-  std::mutex m_slice_mut;
+  std::mutex              m_slice_mut;
   std::condition_variable m_slice_cond;
-  std::vector<bool> m_slice_free;
+  std::vector<bool>       m_slice_free;
 
   // Threads transposing data
   std::vector<std::thread> m_transpose_threads;
 
   // Array to store the number of banks per bank type
   mutable std::array<unsigned int, NBankTypes> m_mfp_count;
-  mutable bool m_sizes_known = false;
+  mutable bool                                 m_sizes_known = false;
 
   // Run and event numbers present in each slice
-  std::vector<EventIDs> m_event_ids;
+  std::vector<EventIDs>          m_event_ids;
   std::vector<std::vector<char>> m_event_masks;
 
   // Storage for the currently open input file
@@ -276,54 +259,56 @@ private:
   AllenConfiguration const* m_allenConfig;
 
   // MBM variables
-  std::vector<BMID> m_bmIDs;
-  size_t n_buffers() const { return std::get<0>(m_bufferConfig.value()); }
+  std::vector<BMID>        m_bmIDs;
+  size_t                   n_buffers() const { return std::get<0>( m_bufferConfig.value() ); }
   std::vector<char const*> m_registered_buffers;
 
   size_t n_receivers() const { return m_receivers.size(); }
 
   std::unordered_set<BankTypes> m_bank_types;
 
-  Gaudi::Property<size_t> m_nslices {this, "NSlices", 6};
-  Gaudi::Property<size_t> m_events_per_slice {this, "EventsPerSlice", 1000};
-  Gaudi::Property<std::vector<std::string>> m_connections {this, "Connections"};
+  Gaudi::Property<size_t>                   m_nslices{this, "NSlices", 6};
+  Gaudi::Property<size_t>                   m_events_per_slice{this, "EventsPerSlice", 1000};
+  Gaudi::Property<std::vector<std::string>> m_connections{this, "Connections"};
 
-  Gaudi::Property<MEP::ProviderSource> m_source {this, "Source", MEP::ProviderSource::MBM};
-  Gaudi::Property<MEP::MBMCom> m_mbm_com {this, "MBMComMethod", MEP::MBMCom::FIFO};
+  Gaudi::Property<MEP::ProviderSource> m_source{this, "Source", MEP::ProviderSource::MBM};
+  Gaudi::Property<MEP::MBMCom>         m_mbm_com{this, "MBMComMethod", MEP::MBMCom::FIFO};
 
   // number of prefetch buffers and transpose threads
-  Gaudi::Property<std::pair<int, int>> m_bufferConfig {this, "BufferConfig", {8, 5},
-    [this](auto &) -> void {
-      // Sanity check on the number of buffers and threads
-      auto [n_buf, n_transpose] = m_bufferConfig.value();
-      if (n_buf < 2) {
-        warning() << "Too few read buffers requested, setting it to 2" << endmsg;
-        n_buf = 2;
-      }
-
-      if (n_transpose > n_buf - 1) {
-        warning() << "Too many transpose threads requested with respect "
-                  << "to the number of read buffers; reducing the number of threads to "
-                  << n_buf - 1 << endmsg;
-        n_transpose = n_buf - 1;
-      }
-      m_bufferConfig.set({n_buf, n_transpose});
-    }, Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
-  Gaudi::Property<int> m_window_size {this, "MPIWindow", 4};
-  Gaudi::Property<bool> m_non_stop {this, "LoopOnMEPs", false};
-  Gaudi::Property<bool> m_preload {this, "Preload", false};
-  Gaudi::Property<long> m_nevents {this, "EvtMax", -1};
-  Gaudi::Property<bool> m_transpose_mep {this, "TransposeMEPs", false};
-  Gaudi::Property<bool> m_split_by_run {this, "SplitByRun", true};
+  Gaudi::Property<std::pair<int, int>> m_bufferConfig{
+      this,
+      "BufferConfig",
+      {8, 5},
+      [this]( auto& ) -> void {
+        // Sanity check on the number of buffers and threads
+        auto [n_buf, n_transpose] = m_bufferConfig.value();
+        if ( n_buf < 2 ) {
+          warning() << "Too few read buffers requested, setting it to 2" << endmsg;
+          n_buf = 2;
+        }
+
+        if ( n_transpose > n_buf - 1 ) {
+          warning() << "Too many transpose threads requested with respect "
+                    << "to the number of read buffers; reducing the number of threads to " << n_buf - 1 << endmsg;
+          n_transpose = n_buf - 1;
+        }
+        m_bufferConfig.set( {n_buf, n_transpose} );
+      },
+      Gaudi::Details::Property::ImmediatelyInvokeHandler{true}};
+  Gaudi::Property<int>  m_window_size{this, "MPIWindow", 4};
+  Gaudi::Property<bool> m_non_stop{this, "LoopOnMEPs", false};
+  Gaudi::Property<bool> m_preload{this, "Preload", false};
+  Gaudi::Property<long> m_nevents{this, "EvtMax", -1};
+  Gaudi::Property<bool> m_transpose_mep{this, "TransposeMEPs", false};
+  Gaudi::Property<bool> m_split_by_run{this, "SplitByRun", true};
 
   // Mapping of receiver card to MPI rank to receive from
-  Gaudi::Property<std::map<std::string, int>> m_receivers {this, "Receivers", {}};
-  Gaudi::Property<std::vector<int>> m_buffer_numa {this, "BufferNUMA", {}};
-  Gaudi::Property<std::vector<std::string>> m_requests {this, "Requests", {}};
-  Gaudi::Property<bool> m_thread_per_buffer {this, "ThreadPerBuffer", true};
+  Gaudi::Property<std::map<std::string, int>> m_receivers{this, "Receivers", {}};
+  Gaudi::Property<std::vector<int>>           m_buffer_numa{this, "BufferNUMA", {}};
+  Gaudi::Property<std::vector<std::string>>   m_requests{this, "Requests", {}};
+  Gaudi::Property<bool>                       m_thread_per_buffer{this, "ThreadPerBuffer", true};
 
   std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mepsInput;
   std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsInput;
   std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbInput;
-
 };
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index 015b0b439..960da6b04 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -1,24 +1,24 @@
 /*****************************************************************************\
 * (c) Copyright 2018-2020 CERN for the benefit of the LHCb Collaboration      *
 \*****************************************************************************/
-#include <unistd.h>
-#include <sys/types.h>
-#include <sys/stat.h>
 #include <fcntl.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+#include <unistd.h>
 
-#include <iostream>
-#include <vector>
-#include <cstring>
 #include <cassert>
+#include <cstring>
+#include <iostream>
 #include <limits>
+#include <vector>
 
 #include <gsl/gsl>
 
+#include <Allen/BankTypes.h>
 #include <EventBuilding/MEP_tools.hpp>
 #include <EventBuilding/MFP_tools.hpp>
 #include <MDF/MDFHeader.h>
 #include <MDF/PosixIO.h>
-#include <Allen/BankTypes.h>
 
 #include <AllenOnline/ReadMEP.h>
 
@@ -36,26 +36,24 @@ namespace {
  * @return     (eof, success, mep_header, span of mep data)
  */
 std::tuple<bool, bool, EB::MEP const*, unsigned, gsl::span<char const>>
-MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
-{
+MEP::read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer ) {
   // Allocate space for the first few words of the MEP header
-  buffer.resize(sizeof(EB::MEP_header));
+  buffer.resize( sizeof( EB::MEP_header ) );
 
   // Read the first few words of the MEP header
   // Why is the number of bytes returned as an int...
-  int read_sc = input.read(&buffer[0], sizeof(EB::MEP_header));
-  if (read_sc == 0) {
+  int read_sc = input.read( &buffer[0], sizeof( EB::MEP_header ) );
+  if ( read_sc == 0 ) {
     cout << "Cannot read more data (Header). End-of-File reached.\n";
     return {true, false, nullptr, 0u, {}};
-  }
-  else if (read_sc < 0) {
-    cerr << "Failed to read header " << strerror(errno) << "\n";
+  } else if ( read_sc < 0 ) {
+    cerr << "Failed to read header " << strerror( errno ) << "\n";
     return {false, false, nullptr, 0u, {}};
   }
 
   // Check magic pattern
-  EB::MEP const* mep = reinterpret_cast<EB::MEP const*>(buffer.data());
-  if (!mep->is_magic_valid()) {
+  EB::MEP const* mep = reinterpret_cast<EB::MEP const*>( buffer.data() );
+  if ( !mep->is_magic_valid() ) {
     cerr << "Wrong magic pattern in MEP header: 0x" << std::hex << mep->header.magic << "\n";
     return {false, false, nullptr, 0u, {}};
   }
@@ -64,19 +62,19 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   // (don't forget to redo the pointers in case the memory was
   // reallocated elsewhere)
   size_t data_size = mep->bytes();
-  buffer.resize(data_size);
+  buffer.resize( data_size );
   char* mep_buffer = &buffer[0];
-  mep = reinterpret_cast<EB::MEP const*>(mep_buffer);
+  mep              = reinterpret_cast<EB::MEP const*>( mep_buffer );
 
-  char* pos = mep_buffer + sizeof(EB::MEP_header);
-  size_t remaining = data_size - sizeof(EB::MEP_header);
-  while (remaining > 0) {
-    size_t chunk = std::min(remaining, static_cast<size_t>(std::numeric_limits<int>::max() / 2));
+  char*  pos       = mep_buffer + sizeof( EB::MEP_header );
+  size_t remaining = data_size - sizeof( EB::MEP_header );
+  while ( remaining > 0 ) {
+    size_t chunk = std::min( remaining, static_cast<size_t>( std::numeric_limits<int>::max() / 2 ) );
     // Read the next chunk
     cout << "Reading " << chunk << " bytes\n";
-    read_sc = input.read(pos, static_cast<int>(chunk));
-    if (read_sc <= 0) {
-      cerr << "Failed to read MEP" << strerror(errno) << "\n";
+    read_sc = input.read( pos, static_cast<int>( chunk ) );
+    if ( read_sc <= 0 ) {
+      cerr << "Failed to read MEP" << strerror( errno ) << "\n";
       return {false, false, nullptr, 0u, {}};
     }
     remaining -= chunk;
@@ -84,8 +82,7 @@ MEP::read_mep(LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer)
   }
 
   // Get the packing factor
-  auto const* mfp = reinterpret_cast<EB::MFP const*>(mep->at(0));
+  auto const* mfp = reinterpret_cast<EB::MFP const*>( mep->at( 0 ) );
 
-  return {false, true, mep, mfp->header.n_banks,
-          {buffer.data(), data_size}};
+  return {false, true, mep, mfp->header.n_banks, {buffer.data(), data_size}};
 }
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index 852a818b6..dfa4a0124 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -4,46 +4,47 @@
 #include <cassert>
 #include <cstring>
 
-#include <Event/ODIN.h>
 #include <Allen/sourceid.h>
+#include <Event/ODIN.h>
 
 #include <AllenOnline/TransposeMEP.h>
 
 namespace {
-  LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes, const uint8_t version ) {
-    LHCb::span<std::uint32_t const> odin_span{reinterpret_cast<std::uint32_t const*>(odin_data + offset), size_bytes / sizeof(uint32_t)};
-    if (version == 6) {
-      return LHCb::ODIN::from_version<6>(odin_span);
+  LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes,
+                          const uint8_t version ) {
+    LHCb::span<std::uint32_t const> odin_span{reinterpret_cast<std::uint32_t const*>( odin_data + offset ),
+                                              size_bytes / sizeof( uint32_t )};
+    if ( version == 6 ) {
+      return LHCb::ODIN::from_version<6>( odin_span );
     } else {
       return LHCb::ODIN{odin_span};
     }
   }
 
-  BankTypes source_id_type(uint16_t src_id) {
-    auto const sd = SourceId_sys(src_id);
-    auto it = Allen::subdetectors.find(static_cast<SourceIdSys>(sd));
+  BankTypes source_id_type( uint16_t src_id ) {
+    auto const sd = SourceId_sys( src_id );
+    auto       it = Allen::subdetectors.find( static_cast<SourceIdSys>( sd ) );
     return it == Allen::subdetectors.end() ? BankTypes::Unknown : it->second;
   }
 
-  template<typename T>
-  T* event_entries(gsl::span<unsigned int> offsets, unsigned const event) {
-    return reinterpret_cast<T*>(&offsets[0]) + offsets[event];
+  template <typename T>
+  T* event_entries( gsl::span<unsigned int> offsets, unsigned const event ) {
+    return reinterpret_cast<T*>( &offsets[0] ) + offsets[event];
   }
 } // namespace
 
-
 std::tuple<bool, std::array<unsigned int, NBankTypes>, std::array<int, NBankTypes>>
 MEP::fill_counts( EB::MEP const* mep ) {
 
   std::array<unsigned, NBankTypes> count{0};
-  std::array<int, NBankTypes> versions{0};
+  std::array<int, NBankTypes>      versions{0};
   for ( size_t i = 0; i < mep->header.n_MFPs; ++i ) {
-    auto const* mfp = mep->at(i);
-    const int source_id = mfp->header.src_id;
-    auto const allen_type = source_id_type(source_id);
-    if (allen_type != BankTypes::Unknown) {
-      auto const sd_index = to_integral(allen_type);
-      versions[sd_index] = mfp->header.block_version;
+    auto const* mfp        = mep->at( i );
+    const int   source_id  = mfp->header.src_id;
+    auto const  allen_type = source_id_type( source_id );
+    if ( allen_type != BankTypes::Unknown ) {
+      auto const sd_index = to_integral( allen_type );
+      versions[sd_index]  = mfp->header.block_version;
       ++count[sd_index];
     }
   }
@@ -56,15 +57,13 @@ void MEP::find_blocks( EB::MEP const* mep, Blocks& blocks ) {
   Blocks tmp{blocks};
   for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) {
     // block offsets are in 4-byte words with respect to the start of the MEP header
-    tmp[i_block] = MEP::Block{mep->at(i_block)};
+    tmp[i_block] = MEP::Block{mep->at( i_block )};
   }
 
   auto const*         src_ids = mep->header.src_ids();
   std::vector<size_t> perm( tmp.size() );
   std::iota( perm.begin(), perm.end(), 0U );
-  std::sort( perm.begin(), perm.end(), [src_ids]( size_t a, size_t b ) {
-    return src_ids[a] < src_ids[b];
-    });
+  std::sort( perm.begin(), perm.end(), [src_ids]( size_t a, size_t b ) { return src_ids[a] < src_ids[b]; } );
 
   for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) { blocks[i_block] = tmp[perm[i_block]]; }
 }
@@ -76,20 +75,20 @@ void MEP::fragment_offsets( MEP::Blocks const& blocks, MEP::SourceOffsets& offse
 
   // Loop over all bank sizes in all blocks
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto const& block = blocks[i_block];
-    auto const align = block.header->align;
-    auto&       o                          = offsets[i_block];
-    uint32_t    fragment_offset            = 0;
+    auto const& block           = blocks[i_block];
+    auto const  align           = block.header->align;
+    auto&       o               = offsets[i_block];
+    uint32_t    fragment_offset = 0;
     for ( size_t i = 0; i < block.header->n_banks; ++i ) {
-      o[i] = fragment_offset;
+      o[i]            = fragment_offset;
       auto const size = block.bank_sizes[i];
-      fragment_offset += (size + EB::get_padding(size, 1 << align));
+      fragment_offset += ( size + EB::get_padding( size, 1 << align ) );
     }
   }
 }
 
 size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
-                           std::unordered_set<BankTypes> const& bank_types,
+                           std::unordered_set<BankTypes> const&        bank_types,
                            std::array<unsigned int, NBankTypes> const& mfp_count, MEP::Blocks const& blocks,
                            MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval,
                            bool split_by_run ) {
@@ -98,13 +97,13 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
 
   // Loop over all bank sizes in all blocks
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
-    auto&       source_offsets = input_offsets[i_block];
+    auto& source_offsets = input_offsets[i_block];
 
-    auto const& block = blocks[i_block];
-    auto const* bank_sizes     = block.bank_sizes;
-    auto const  allen_type     = source_id_type(block.header->src_id);
-    auto const  sd_index       = to_integral(allen_type);
-    unsigned    run_number     = 0;
+    auto const& block      = blocks[i_block];
+    auto const* bank_sizes = block.bank_sizes;
+    auto const  allen_type = source_id_type( block.header->src_id );
+    auto const  sd_index   = to_integral( allen_type );
+    unsigned    run_number = 0;
 
     if ( bank_types.count( allen_type ) ) {
       auto& event_offsets = slices[sd_index][slice_index].offsets;
@@ -113,10 +112,9 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
         // First check for run changes in ODIN banks
         if ( split_by_run && allen_type == BankTypes::ODIN ) {
           // decode ODIN banks to check for run changes
-          if (block.bank_types[i] != LHCb::RawBank::ODIN) continue;
+          if ( block.bank_types[i] != LHCb::RawBank::ODIN ) continue;
 
-          auto const odin = decode_odin(block.payload, source_offsets[i],
-                                        bank_sizes[i], block.header->block_version);
+          auto const odin = decode_odin( block.payload, source_offsets[i], bank_sizes[i], block.header->block_version );
           unsigned const odin_run_number = odin.runNumber();
 
           // if splitting by run, check all events have same run number
@@ -129,9 +127,9 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
         }
 
         // Anticipate offset structure already here, i.e. don't assign to the first one
-        auto  idx           = i - event_start + 1;
+        auto idx = i - event_start + 1;
         // Allen raw bank format has the sourceID followed by the raw bank data.
-        event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i] + EB::get_padding(bank_sizes[i], sizeof(unsigned));
+        event_offsets[idx] += sizeof( uint32_t ) + bank_sizes[i] + EB::get_padding( bank_sizes[i], sizeof( unsigned ) );
       }
     }
   }
@@ -140,8 +138,8 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
   // "Allen" offsets per bank type per event
   size_t n_frag = ( event_end - event_start );
   for ( auto allen_type : bank_types ) {
-    auto const sd_index = to_integral(allen_type);
-    auto& slice = slices[sd_index][slice_index];
+    auto const sd_index = to_integral( allen_type );
+    auto&      slice    = slices[sd_index][slice_index];
     slice.offsets[0]    = 0;
     auto preamble_words = 2 + mfp_count[sd_index];
     for ( size_t i = 1; i <= ( event_end - event_start ) && i <= n_frag; ++i ) {
@@ -160,30 +158,30 @@ size_t MEP::allen_offsets( Allen::Slices& slices, int const slice_index,
 
   // Set offsets_size here to make sure it's consistent with the max
   for ( auto allen_type : bank_types ) {
-    auto const sd_index = to_integral(allen_type);
+    auto const sd_index                     = to_integral( allen_type );
     slices[sd_index][slice_index].n_offsets = n_frag + 1;
   }
   return n_frag;
 }
 
 std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int const slice_index,
-                                                 std::unordered_set<BankTypes> const& subdetectors,
+                                                 std::unordered_set<BankTypes> const&        subdetectors,
                                                  std::array<unsigned int, NBankTypes> const& mfp_count,
                                                  EventIDs& event_ids, std::vector<char>& event_mask,
-                                                 MEP::Blocks const& blocks,
-                                                 std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
+                                                 MEP::Blocks const& blocks, std::tuple<size_t, size_t> const& interval,
+                                                 bool split_by_run ) {
 
   auto [event_start, event_end] = interval;
 
-  BankTypes     prev_type   = BankTypes::Unknown;
-  size_t        offset_index = 0;
-  std::optional<unsigned> run_number = std::nullopt;
+  BankTypes               prev_type    = BankTypes::Unknown;
+  size_t                  offset_index = 0;
+  std::optional<unsigned> run_number   = std::nullopt;
 
   for ( size_t i_block = 0; i_block < blocks.size(); ++i_block ) {
     auto const& block      = blocks[i_block];
     auto const  source_id  = block.header->src_id;
-    auto const  allen_type = source_id_type(source_id);
-    auto const  sd_index   = to_integral(allen_type);
+    auto const  allen_type = source_id_type( source_id );
+    auto const  sd_index   = to_integral( allen_type );
     auto        n_blocks   = mfp_count[sd_index];
     auto const* bank_sizes = block.bank_sizes;
     auto const* bank_types = block.bank_types;
@@ -194,12 +192,12 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       // decode ODIN bank to obtain run and event numbers
       unsigned fragment_offset = 0;
       for ( unsigned i_event = 0; i_event < event_end; ++i_event ) {
-        if ( i_event >= event_start) {
-          if (bank_types[i_event] == LHCb::RawBank::ODIN) {
-            auto const odin = decode_odin(block.payload, fragment_offset,
-                                          bank_sizes[i_event], block.header->block_version);
+        if ( i_event >= event_start ) {
+          if ( bank_types[i_event] == LHCb::RawBank::ODIN ) {
+            auto const odin =
+                decode_odin( block.payload, fragment_offset, bank_sizes[i_event], block.header->block_version );
             unsigned const odin_run_number = odin.runNumber();
-            auto const event_number = odin.eventNumber();
+            auto const     event_number    = odin.eventNumber();
 
             // if splitting by run, check all events have same run number
             if ( !run_number ) {
@@ -210,32 +208,31 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
             }
             event_ids.emplace_back( odin_run_number, event_number );
             event_mask[i_event - event_start] = 1;
-          }
-          else {
+          } else {
             event_ids.emplace_back( 0, 0 );
             event_mask[i_event - event_start] = 0;
           }
         }
-        fragment_offset += bank_sizes[i_event] + EB::get_padding(bank_sizes[i_event], 1 << align);
+        fragment_offset += bank_sizes[i_event] + EB::get_padding( bank_sizes[i_event], 1 << align );
       }
     }
 
     if ( subdetectors.count( allen_type ) ) {
-      auto& slice = slices[sd_index][slice_index];
-      auto& fragments = slice.fragments;
-      auto& offsets = slice.offsets;
-      auto& offsets_size = slice.n_offsets;
+      auto& slice         = slices[sd_index][slice_index];
+      auto& fragments     = slice.fragments;
+      auto& offsets       = slice.offsets;
+      auto& offsets_size  = slice.n_offsets;
       auto& sizes_offsets = slice.sizes;
       auto& types_offsets = slice.types;
 
       // Calculate block offset and size
       size_t interval_offset = 0, interval_size = 0;
       for ( size_t i = 0; i < event_start; ++i ) {
-        auto const padded_size = bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
+        auto const padded_size = bank_sizes[i] + EB::get_padding( bank_sizes[i], 1 << align );
         interval_offset += padded_size;
       }
       for ( size_t i = event_start; i < event_end; ++i ) {
-        auto const padded_size = bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
+        auto const padded_size = bank_sizes[i] + EB::get_padding( bank_sizes[i], 1 << align );
         interval_size += padded_size;
       }
 
@@ -246,22 +243,22 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
         offsets[2 + n_blocks] = 0;
         offset_index          = 0;
 
-        sizes_offsets[offset_index] = 2 * mfp_count[to_integral(allen_type)];
-        types_offsets[offset_index] = 4 * mfp_count[to_integral(allen_type)];
+        sizes_offsets[offset_index] = 2 * mfp_count[to_integral( allen_type )];
+        types_offsets[offset_index] = 4 * mfp_count[to_integral( allen_type )];
 
-        prev_type             = allen_type;
+        prev_type = allen_type;
       } else {
         sizes_offsets[offset_index] = sizes_offsets[offset_index - 1] + event_end - event_start;
         types_offsets[offset_index] = types_offsets[offset_index - 1] + event_end - event_start;
       }
 
       // Copy bank sizes
-      auto* sizes = event_entries<uint16_t>(sizes_offsets, offset_index);
-      std::memcpy(sizes, bank_sizes + event_start, (event_end - event_start) * sizeof(uint16_t));
+      auto* sizes = event_entries<uint16_t>( sizes_offsets, offset_index );
+      std::memcpy( sizes, bank_sizes + event_start, ( event_end - event_start ) * sizeof( uint16_t ) );
 
       // Copy bank types
-      auto* types = event_entries<uint8_t>(types_offsets, offset_index);
-      std::memcpy(types, bank_types + event_start, (event_end - event_start));
+      auto* types = event_entries<uint8_t>( types_offsets, offset_index );
+      std::memcpy( types, bank_types + event_start, ( event_end - event_start ) );
 
       // Store source ID
       offsets[2 + offset_index] = source_id;
@@ -274,9 +271,9 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       // Fill fragment offsets
       size_t oi = 0, idx = 0;
       for ( size_t i = event_start; i < event_end; ++i ) {
-        idx               = i - event_start + 1;
-        oi                = 2 + n_blocks * ( 1 + idx ) + offset_index;
-        offsets[oi] = offsets[oi - n_blocks] + bank_sizes[i] + EB::get_padding(bank_sizes[i], 1 << align);
+        idx         = i - event_start + 1;
+        oi          = 2 + n_blocks * ( 1 + idx ) + offset_index;
+        offsets[oi] = offsets[oi - n_blocks] + bank_sizes[i] + EB::get_padding( bank_sizes[i], 1 << align );
       }
       // Update offsets_size
       offsets_size = oi;
@@ -292,21 +289,20 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
 }
 
 bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
-                         std::unordered_set<BankTypes> const& subdetectors,
+                         std::unordered_set<BankTypes> const&        subdetectors,
                          std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
-                         std::vector<char>& event_mask,
-                         EB::MEP const* mep, MEP::Blocks const& blocks,
+                         std::vector<char>& event_mask, EB::MEP const* mep, MEP::Blocks const& blocks,
                          MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval ) {
   auto [start_event, end_event] = interval;
 
   // Loop over all bank data of this event
   size_t bank_index = 1;
-  auto prev_type = BankTypes::Unknown;
+  auto   prev_type  = BankTypes::Unknown;
 
   for ( size_t i_block = 0; i_block < mep->header.n_MFPs; ++i_block ) {
     auto const& block          = blocks[i_block];
     auto const  source_id      = block.header->src_id;
-    auto        allen_type     = source_id_type(source_id);
+    auto        allen_type     = source_id_type( source_id );
     auto&       source_offsets = input_offsets[i_block];
     auto const* bank_sizes     = block.bank_sizes;
     auto const* bank_types     = block.bank_types;
@@ -315,13 +311,12 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
     if ( allen_type == BankTypes::ODIN ) {
       // decode ODIN bank to obtain run and event numbers
       for ( uint16_t i_event = start_event; i_event < end_event; ++i_event ) {
-        if (bank_types[i_event] != LHCb::RawBank::ODIN) {
+        if ( bank_types[i_event] != LHCb::RawBank::ODIN ) {
           event_ids.emplace_back( 0, 0 );
           event_mask[i_event - start_event] = 0;
-        }
-        else {
-          auto const odin = decode_odin(block.payload, source_offsets[i_event],
-                                        bank_sizes[i_event], block.header->block_version);
+        } else {
+          auto const odin =
+              decode_odin( block.payload, source_offsets[i_event], bank_sizes[i_event], block.header->block_version );
           event_ids.emplace_back( odin.runNumber(), odin.eventNumber() );
           event_mask[i_event - start_event] = 1;
         }
@@ -331,7 +326,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
     if ( !subdetectors.count( allen_type ) ) {
       prev_type = allen_type;
     } else {
-      auto        sd_index      = to_integral(allen_type);
+      auto        sd_index      = to_integral( allen_type );
       auto&       transposed    = slices[sd_index][slice_index].fragments[0];
       auto const& event_offsets = slices[sd_index][slice_index].offsets;
       auto&       sizes_offsets = slices[sd_index][slice_index].sizes;
@@ -340,19 +335,19 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
       auto const n_events = end_event - start_event;
 
       if ( allen_type != prev_type ) {
-        bank_index = 1;
+        bank_index       = 1;
         auto n_sd_blocks = mfp_count[sd_index];
 
-        for (size_t i = 0; i < n_events; ++i) {
-          sizes_offsets[i] = 2 * (end_event - start_event + 1) + i * n_sd_blocks;
-          types_offsets[i] = 4 * (end_event - start_event + 1) + i * n_sd_blocks;
+        for ( size_t i = 0; i < n_events; ++i ) {
+          sizes_offsets[i] = 2 * ( end_event - start_event + 1 ) + i * n_sd_blocks;
+          types_offsets[i] = 4 * ( end_event - start_event + 1 ) + i * n_sd_blocks;
         }
-        prev_type  = allen_type;
+        prev_type = allen_type;
       }
 
-      for (size_t i = 0; i < n_events; ++i) {
-        event_entries<uint16_t>(sizes_offsets, i)[bank_index - 1] = bank_sizes[start_event + i];
-        event_entries<uint8_t>(types_offsets, i)[bank_index - 1] = bank_types[start_event + i];
+      for ( size_t i = 0; i < n_events; ++i ) {
+        event_entries<uint16_t>( sizes_offsets, i )[bank_index - 1] = bank_sizes[start_event + i];
+        event_entries<uint8_t>( types_offsets, i )[bank_index - 1]  = bank_types[start_event + i];
       }
 
       for ( size_t i_event = start_event; i_event < end_event && i_event < block.header->n_banks; ++i_event ) {
@@ -377,8 +372,8 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
         }
 
         // get offset for this bank and store offset for next bank
-        auto offset                     = banks_offsets_write[bank_index - 1];
-        auto frag_size                  = bank_sizes[i_event] + EB::get_padding(bank_sizes[i_event], sizeof(unsigned));
+        auto offset    = banks_offsets_write[bank_index - 1];
+        auto frag_size = bank_sizes[i_event] + EB::get_padding( bank_sizes[i_event], sizeof( unsigned ) );
         banks_offsets_write[bank_index] = offset + frag_size + sizeof( uint32_t );
 
         // Where to write the bank data itself
@@ -398,20 +393,22 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
   return true;
 }
 
-std::tuple<bool, bool, size_t> MEP::transpose_events(
-    Allen::Slices& slices, int const slice_index,
-    std::unordered_set<BankTypes> const& bank_types, std::array<unsigned int, NBankTypes> const& mfp_count,
-    EventIDs& event_ids, std::vector<char>& event_mask, EB::MEP const* mep, MEP::Blocks const& blocks,
-    MEP::SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
+std::tuple<bool, bool, size_t> MEP::transpose_events( Allen::Slices& slices, int const slice_index,
+                                                      std::unordered_set<BankTypes> const&        bank_types,
+                                                      std::array<unsigned int, NBankTypes> const& mfp_count,
+                                                      EventIDs& event_ids, std::vector<char>& event_mask,
+                                                      EB::MEP const* mep, MEP::Blocks const& blocks,
+                                                      MEP::SourceOffsets const&         source_offsets,
+                                                      std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
   auto [event_start, event_end] = interval;
 
   bool success = true;
 
-  auto to_transpose = allen_offsets( slices, slice_index, bank_types, mfp_count, blocks, source_offsets,
-                                     interval, split_by_run );
+  auto to_transpose =
+      allen_offsets( slices, slice_index, bank_types, mfp_count, blocks, source_offsets, interval, split_by_run );
 
-  transpose_mep( slices, slice_index, bank_types, mfp_count, event_ids, event_mask, mep, blocks,
-                 source_offsets, {event_start, event_start + to_transpose} );
+  transpose_mep( slices, slice_index, bank_types, mfp_count, event_ids, event_mask, mep, blocks, source_offsets,
+                 {event_start, event_start + to_transpose} );
 
   return {success, to_transpose != ( event_end - event_start ), to_transpose};
 }
-- 
GitLab


From efcfcec4f1d010dc223e236daa0c3ee60fb135a4 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 22 Apr 2022 23:45:23 +0200
Subject: [PATCH 093/120] Update allen_read_mep to allow dumping of individual
 banks

---
 AllenOnline/CMakeLists.txt           |   1 +
 AllenOnline/application/read_mep.cpp | 114 +++++++++++++++++++++++----
 2 files changed, 100 insertions(+), 15 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 4075e24be..554d8c91a 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -67,6 +67,7 @@ gaudi_add_executable(allen_test_host_register
       Online::OnlineBase
       Allen::Backend
       LHCb::ZMQLib
+      Boost::program_options
     TEST
 )
 
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 3916aa36a..4fd609d17 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -15,6 +15,11 @@
 #include <sys/types.h>
 #include <unistd.h>
 
+#include <boost/algorithm/string.hpp>
+#include <boost/format.hpp>
+#include <boost/lexical_cast.hpp>
+#include <boost/program_options.hpp>
+
 #include <Event/ODIN.h>
 #include <Event/RawBank.h>
 
@@ -33,6 +38,8 @@
 #include <AllenOnline/TransposeMEP.h>
 
 using namespace std;
+namespace po = boost::program_options;
+namespace ba = boost::algorithm;
 
 struct CaloRawBank {
   uint32_t        source_id = 0;
@@ -74,13 +81,63 @@ std::vector<char> contiguous_mfps( Allen::Slice const& mep_data ) {
 }
 
 int main( int argc, char* argv[] ) {
-  if ( argc != 3 ) {
-    cout << "usage: read_mep file.mep n_mep" << endl;
-    return -1;
+
+  string filename;
+  string dump;
+  size_t n_meps = 0;
+  size_t n_skip = 0;
+
+  // Declare the supported options.
+  po::options_description desc( "Allowed options" );
+  desc.add_options()( "help,h", "produce help message" )( "filename,f", po::value<string>( &filename ),
+                                                          "filename pattern" )(
+      "n_meps,n", po::value<size_t>( &n_meps ),
+      "number of events" )( "skip,s", po::value<size_t>( &n_skip )->default_value( 0 ), "number of events to skip" )(
+      "dump", po::value<string>( &dump ), "dump bank content (source_id_type,start_event[,end_event],bank_number" );
+
+  po::positional_options_description p;
+  p.add( "filename", 1 );
+  p.add( "n_meps", 1 );
+
+  po::variables_map vm;
+  po::store( po::command_line_parser( argc, argv ).options( desc ).positional( p ).run(), vm );
+  po::notify( vm );
+
+  if ( vm.count( "help" ) ) {
+    std::cout << desc << "\n";
+    return 1;
   }
 
-  string filename = {argv[1]};
-  size_t n_meps   = atol( argv[2] );
+  std::string dump_type;
+  unsigned    dump_start = 0, dump_end = 0;
+  unsigned    dump_bank = 0;
+  if ( !dump.empty() ) {
+    vector<string> entries;
+    ba::split( entries, dump, boost::is_any_of( "," ) );
+
+    if ( entries.size() != 3 && entries.size() != 4 ) {
+      cout << "Invalid dump specification: " << dump << "\n";
+      return 1;
+    }
+
+    for ( uint16_t i = 0; i < static_cast<uint16_t>( SourceIdSys::SourceIdSys_TDET ); ++i ) {
+      auto const* dt = SourceId_sysstr( i << 11 );
+      if ( dt != nullptr && entries[0] == std::string{dt} ) { dump_type = dt; }
+    }
+    if ( dump_type.empty() ) {
+      cout << "Invalid source type: " << entries[0] << "\n";
+      return 1;
+    }
+
+    dump_start = boost::lexical_cast<unsigned>( entries[1] );
+    if ( entries.size() == 3 ) {
+      dump_end  = dump_start + 1;
+      dump_bank = boost::lexical_cast<unsigned>( entries[2] );
+    } else {
+      dump_end  = boost::lexical_cast<unsigned>( entries[2] );
+      dump_bank = boost::lexical_cast<unsigned>( entries[3] );
+    }
+  }
 
   // Some storage for reading the events into
   bool eof = false, success = false;
@@ -127,7 +184,12 @@ int main( int argc, char* argv[] ) {
       for ( auto& offsets : slice.offsets ) { offsets.resize( slice.packing_factor + 1 ); }
     }
 
+    if ( n_skip != 0 && n_skip-- > 0 ) continue;
+
     MEP::find_blocks( mep, slice.blocks );
+
+    MEP::fragment_offsets( slice.blocks, slice.offsets );
+
     auto [success, mfp_count, banks_version] = MEP::fill_counts( mep );
 
     if ( i_mep == 0 ) {
@@ -161,7 +223,7 @@ int main( int argc, char* argv[] ) {
     auto const& odin_sizes   = odin_slice.sizes;
     auto const& odin_types   = odin_slice.types;
     auto        odin_bank    = MEP::raw_bank<ODINRawBank>( odin_banks.data(), odin_offsets.data(), odin_sizes.data(),
-                                                 odin_types.data(), 0, 0 );
+                                                 odin_types.data(), dump_start, 0 );
     LHCb::ODIN  odin;
     if ( banks_version[odin_index] == 7 ) {
       odin = LHCb::ODIN{{odin_bank.data, 10}};
@@ -172,26 +234,48 @@ int main( int argc, char* argv[] ) {
          << " event: " << std::setw( 12 ) << odin.eventNumber() << "\n";
 
     // Print block information
-    for ( auto const& block : slice.blocks ) {
+    std::string prev_det;
+    unsigned    block_counter = 0;
+
+    for ( size_t i_block = 0; i_block < slice.blocks.size(); ++i_block ) {
+
+      auto const& block = slice.blocks[i_block];
+
       // block offsets are in number of 4-byte words
       auto const  source_id = block.header->src_id;
       std::string det       = SourceId_sysstr( source_id );
       std::string fill( 7 - det.size(), ' ' );
 
+      if ( prev_det != det ) {
+        block_counter = 0;
+        prev_det      = det;
+      } else {
+        ++block_counter;
+      }
+
       size_t      padded_size = 0;
       auto const* bank_sizes  = block.bank_sizes;
       for ( size_t i = 0; i < block.header->n_banks; ++i ) {
         padded_size += bank_sizes[i] + EB::get_padding( bank_sizes[i], 1 << block.header->align );
       }
 
-      cout << "fragment"
-           << " magic: 0x" << std::hex << block.header->magic << std::dec << " source_id: " << std::setw( 6 )
-           << source_id << " top5: " << std::setw( 2 ) << SourceId_sys( source_id ) << fill << " (" << det << ") "
-           << std::setw( 5 ) << SourceId_num( source_id ) << " version: " << std::setw( 2 )
-           << unsigned{block.header->block_version} << " align: " << std::setw( 3 )
-           << pow( 2, unsigned{block.header->align} ) << " size: " << std::setw( 8 )
-           << block.header->bytes() - block.header->header_size() << " padded_size: " << std::setw( 8 ) << padded_size
-           << "\n";
+      bool dump_block = !dump.empty() && det == dump_type && dump_bank == block_counter;
+
+      if ( dump.empty() || dump_block ) {
+        cout << "MFP"
+             << " magic: 0x" << std::hex << block.header->magic << std::dec << " source_id: " << std::setw( 6 )
+             << source_id << " top5: " << std::setw( 2 ) << SourceId_sys( source_id ) << fill << " (" << det << ") "
+             << std::setw( 5 ) << SourceId_num( source_id ) << " version: " << std::setw( 2 )
+             << unsigned{block.header->block_version} << " align: " << std::setw( 3 )
+             << pow( 2, unsigned{block.header->align} ) << " size: " << std::setw( 8 )
+             << block.header->bytes() - block.header->header_size() << " padded_size: " << std::setw( 8 ) << padded_size
+             << "\n";
+      }
+      if ( dump_block ) {
+        for ( unsigned evt = dump_start; evt < dump_end; ++evt ) {
+          MDF::dump_hex( block.payload + slice.offsets[i_block][evt], block.bank_sizes[evt] );
+        }
+      }
     }
   }
   return 0;
-- 
GitLab


From 056deed0cc77854700d344a74340530ac317a1b2 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 26 Apr 2022 15:54:42 +0200
Subject: [PATCH 094/120] Add tests to AllenOnline

- Process 2000 events from a MEP in slices 500 and write an output MDF
- check that all banks are indentical between the MEP and output MDF
- check also if the MEP is transposed
- run CPU decoding algorithms (including DecReports and SelReprots)
  on the output MDF
- Reinstate the 10 event test of allen.py with MEP input

A property has been added to the MEPProvider to mask the five most
significant bits of the source IDs. This is required to test
downstream of Allen until new simulation is available.

In addition: don't add -ccbin to nvcc if CMAKE_CUDA_HOST_COMPILER is
already set
---
 AllenOnline/CMakeLists.txt                    |  5 +-
 AllenOnline/application/MPISend.cpp           |  2 +-
 AllenOnline/application/read_mep.cpp          |  8 ++-
 AllenOnline/include/AllenOnline/ReadMEP.h     |  4 +-
 .../include/AllenOnline/TransposeMEP.h        |  7 ++-
 AllenOnline/src/MEPProvider.cpp               | 52 +++++++++++-----
 AllenOnline/src/MEPProvider.h                 |  3 +-
 AllenOnline/src/ReadMEP.cpp                   | 18 +++---
 AllenOnline/src/TransposeMEP.cpp              | 24 +++----
 AllenOnline/tests/options/allen_output.py     | 22 +++++++
 AllenOnline/tests/options/cpu_decoding.py     | 62 +++++++++++++++++++
 .../tests/qmtest/decode_allen_output.qmt      | 36 +++++++++++
 AllenOnline/tests/qmtest/mep_input.qmt        | 34 ++++++++++
 AllenOnline/tests/qmtest/mep_passthrough.qmt  | 41 ++++++++++++
 AllenOnline/tests/qmtest/test_mep_banks.qmt   | 36 +++++++++++
 .../tests/qmtest/test_mep_banks_transpose.qmt | 37 +++++++++++
 .../tests/refs/decode_allen_output.ref        |  9 +++
 AllenOnline/tests/refs/mep_passthrough.ref    | 38 ++++++++++++
 AllenOnline/tests/refs/test_mep_banks.ref     | 17 +++++
 19 files changed, 407 insertions(+), 48 deletions(-)
 create mode 100644 AllenOnline/tests/options/allen_output.py
 create mode 100644 AllenOnline/tests/options/cpu_decoding.py
 create mode 100644 AllenOnline/tests/qmtest/decode_allen_output.qmt
 create mode 100644 AllenOnline/tests/qmtest/mep_input.qmt
 create mode 100644 AllenOnline/tests/qmtest/mep_passthrough.qmt
 create mode 100644 AllenOnline/tests/qmtest/test_mep_banks.qmt
 create mode 100644 AllenOnline/tests/qmtest/test_mep_banks_transpose.qmt
 create mode 100644 AllenOnline/tests/refs/decode_allen_output.ref
 create mode 100644 AllenOnline/tests/refs/mep_passthrough.ref
 create mode 100644 AllenOnline/tests/refs/test_mep_banks.ref

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 554d8c91a..4ece9f850 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -68,7 +68,6 @@ gaudi_add_executable(allen_test_host_register
       Allen::Backend
       LHCb::ZMQLib
       Boost::program_options
-    TEST
 )
 
 gaudi_add_executable(allen_mpi_send
@@ -98,3 +97,7 @@ if (TARGET MPI::MPI_CXX AND TARGET PkgConfig::hwloc)
     target_compile_definitions(${component} PRIVATE HAVE_MPI)
   endforeach()
 endif()
+
+gaudi_install(PYTHON)
+
+gaudi_add_tests(QMTest)
diff --git a/AllenOnline/application/MPISend.cpp b/AllenOnline/application/MPISend.cpp
index 9b5e6b6b9..8a1dc9cbd 100644
--- a/AllenOnline/application/MPISend.cpp
+++ b/AllenOnline/application/MPISend.cpp
@@ -99,7 +99,7 @@ int main( int argc, char* argv[] ) {
     while ( success && !eof ) {
       std::cout << "." << std::flush;
 
-      std::tie( eof, success, mep, pf, mep_span ) = MEP::read_mep( input, data );
+      std::tie( eof, success, mep, pf, mep_span ) = MEP::read_mep( input, data, std::cout, std::cerr );
       if ( !packing_factor ) {
         packing_factor = pf;
       } else if ( *packing_factor != pf ) {
diff --git a/AllenOnline/application/read_mep.cpp b/AllenOnline/application/read_mep.cpp
index 4fd609d17..1f1130c3b 100644
--- a/AllenOnline/application/read_mep.cpp
+++ b/AllenOnline/application/read_mep.cpp
@@ -168,8 +168,9 @@ int main( int argc, char* argv[] ) {
 
   for ( size_t i_mep = 0; i_mep < n_meps && !eof; ++i_mep ) {
 
-    std::tie( eof, success, slice.mep, slice.packing_factor, slice.mep_data ) = MEP::read_mep( input, data );
-    auto const* mep                                                           = slice.mep;
+    std::tie( eof, success, slice.mep, slice.packing_factor, slice.mep_data ) =
+        MEP::read_mep( input, data, std::cout, std::cerr );
+    auto const* mep = slice.mep;
     if ( !success ) {
       return 1;
     } else {
@@ -243,7 +244,8 @@ int main( int argc, char* argv[] ) {
 
       // block offsets are in number of 4-byte words
       auto const  source_id = block.header->src_id;
-      std::string det       = SourceId_sysstr( source_id );
+      auto const* dt        = SourceId_sysstr( source_id );
+      std::string det       = dt != nullptr ? dt : "Unknown";
       std::string fill( 7 - det.size(), ' ' );
 
       if ( prev_det != det ) {
diff --git a/AllenOnline/include/AllenOnline/ReadMEP.h b/AllenOnline/include/AllenOnline/ReadMEP.h
index 8fa79db8a..81463b251 100644
--- a/AllenOnline/include/AllenOnline/ReadMEP.h
+++ b/AllenOnline/include/AllenOnline/ReadMEP.h
@@ -3,6 +3,7 @@
 \*****************************************************************************/
 #pragma once
 
+#include <iostream>
 #include <vector>
 
 #include <gsl/gsl>
@@ -12,5 +13,6 @@
 
 namespace MEP {
   std::tuple<bool, bool, EB::MEP const*, unsigned, gsl::span<char const>>
-  read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer );
+  read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer, std::ostream& info_stream,
+            std::ostream& error_stream );
 }
diff --git a/AllenOnline/include/AllenOnline/TransposeMEP.h b/AllenOnline/include/AllenOnline/TransposeMEP.h
index f609d2cfd..0bba72116 100644
--- a/AllenOnline/include/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/include/AllenOnline/TransposeMEP.h
@@ -91,7 +91,7 @@ namespace MEP {
                                               std::array<unsigned int, NBankTypes> const& banks_count,
                                               EventIDs& event_ids, std::vector<char>& event_mask,
                                               MEP::Blocks const& blocks, std::tuple<size_t, size_t> const& interval,
-                                              bool split_by_run = false );
+                                              bool split_by_run = false, bool mask_top5 = false );
 
   /**
    * @brief      Transpose events to Allen layout
@@ -107,7 +107,8 @@ namespace MEP {
   bool transpose_mep( Allen::Slices& slices, int const slice_index, std::unordered_set<BankTypes> const& bank_types,
                       std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
                       std::vector<char>& event_mask, EB::MEP const* mep, Blocks const& blocks,
-                      SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval );
+                      SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval,
+                      bool mask_top5 = false );
 
   /**
    * @brief      Transpose MEP to Allen layout
@@ -125,5 +126,5 @@ namespace MEP {
                     std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
                     std::vector<char>& event_mask, EB::MEP const* mep, Blocks const& blocks,
                     SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval,
-                    bool split_by_run = false );
+                    bool split_by_run = false, bool mask_top5 = false );
 } // namespace MEP
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index fded95fb3..59d5d5d05 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -26,6 +26,8 @@
 #include "AllenConfiguration.h"
 #include "MEPProvider.h"
 
+#include <TROOT.h>
+
 #ifdef HAVE_MPI
 #  include <AllenOnline/MPIConfig.h>
 #  include <hwloc.h>
@@ -230,12 +232,14 @@ void MEPProvider::copy_banks( size_t const slice_index, unsigned int const event
 
     assert( ( offset + bank_header_size + Allen::padded_bank_size( fragment_size ) ) <=
             static_cast<size_t>( buffer.size() ) );
-    offset += Allen::add_raw_bank( block.bank_types[mep_event], header->block_version, header->src_id,
+    auto const source_id = m_mask_top5.value() ? header->src_id & 0x7FF : header->src_id;
+    offset += Allen::add_raw_bank( block.bank_types[mep_event], header->block_version, source_id,
                                    {block.payload + fragment_offset, fragment_size}, buffer.data() + offset );
   }
 }
 
 StatusCode MEPProvider::initialize() {
+
   m_slice_free.resize( m_nslices.value(), true );
   m_event_ids.resize( m_nslices.value() );
   m_event_masks.resize( m_nslices.value() );
@@ -352,9 +356,19 @@ StatusCode MEPProvider::start() {
     m_ninput_threads = 1;
     m_input_threads.emplace_back( std::thread{&MEPProvider::mpi_read, this} );
 #else
-    throw StrException{"MPI requested, but no MPI support built in."};
+    error() << "MPI requested, but no MPI support built in." << endmsg;
+    return StatusCode::FAILURE;
 #endif
   } else if ( m_source == MEP::ProviderSource::Files && m_input_threads.empty() ) {
+    auto input = LHCb::StreamDescriptor::bind( *m_current );
+    if ( input.ioDesc > 0 ) {
+      info() << "Opened " << *m_current << endmsg;
+    } else {
+      error() << "Failed to open " << *m_current << " " << strerror( errno ) << endmsg;
+      return StatusCode::FAILURE;
+    }
+    m_inputs.emplace( *m_current, input );
+
     m_ninput_threads = 1;
     m_input_threads.emplace_back( std::thread{&MEPProvider::mep_read, this} );
   } else if ( m_source == MEP::ProviderSource::MBM ) {
@@ -718,16 +732,17 @@ bool MEPProvider::open_file() const {
       }
     }
 
-    if ( m_input ) m_input->close();
-
-    m_input = LHCb::StreamDescriptor::bind( *m_current );
-    if ( m_input->ioDesc > 0 ) {
-      info() << "Opened " << *m_current << endmsg;
-      good = true;
-    } else {
-      error() << "Failed to open " << *m_current << " " << strerror( errno ) << endmsg;
+    auto input = m_inputs.find( *m_current );
+    if ( input == m_inputs.end() ) {
+      error() << "Failed to find IO object for connection " << *m_current << endmsg;
       return false;
+    } else if ( input->second.ioDesc < 0 ) {
+      error() << "IO object for connection " << *m_current << " has not been opened" << endmsg;
+      return false;
+    } else if ( !m_non_stop.value() ) {
+      input->second.close();
     }
+
     ++m_current;
   }
   return good;
@@ -830,7 +845,7 @@ void MEPProvider::mep_read() {
     if ( m_done ) break;
 
     // open the first file
-    if ( !m_input && !open_file() ) {
+    if ( m_current == m_connections.end() && !open_file() ) {
       read_error();
       return;
     }
@@ -861,8 +876,14 @@ void MEPProvider::mep_read() {
 
     while ( !success || eof ) {
       if ( !m_preload.value() || ( preloaded < n_buffers() && !preloaded_buffer[i_buffer] ) ) {
+        auto input = m_inputs.find( *m_current );
+        if ( input == m_inputs.end() ) {
+          error() << "No input for connection " << *m_current << endmsg;
+          read_error();
+          break;
+        }
         std::tie( eof, success, slice.mep, slice.packing_factor, slice.mep_data ) =
-            MEP::read_mep( *m_input, read_buffer );
+            MEP::read_mep( input->second, read_buffer, info().stream(), error().stream() );
         if ( !eof && msgLevel( MSG::DEBUG ) )
           debug() << "Read mep with packing factor " << slice.packing_factor << endmsg;
 
@@ -1381,15 +1402,16 @@ void MEPProvider::transpose( int thread_id ) {
       // Transpose the events into the slice
       std::tie( good, transpose_full, n_transposed ) =
           MEP::transpose_events( m_slices, *slice_index, m_bank_types, m_mfp_count, event_ids, event_mask, slice.mep,
-                                 slice.blocks, slice.offsets, interval, m_split_by_run.value() );
+                                 slice.blocks, slice.offsets, interval, m_split_by_run.value(), m_mask_top5.value() );
       if ( msgLevel( MSG::DEBUG ) ) {
         debug() << "Transpose " << thread_id << ": Transposed slice " << *slice_index << "; good: " << good
                 << "; full: " << transpose_full << "; n_transposed: " << n_transposed << endmsg;
       }
     } else {
       // Calculate fragment offsets in MEP per sub-detector
-      std::tie( good, transpose_full, n_transposed ) = MEP::mep_offsets(
-          m_slices, *slice_index, m_bank_types, m_mfp_count, event_ids, event_mask, slice.blocks, interval );
+      std::tie( good, transpose_full, n_transposed ) =
+          MEP::mep_offsets( m_slices, *slice_index, m_bank_types, m_mfp_count, event_ids, event_mask, slice.blocks,
+                            interval, m_split_by_run.value(), m_mask_top5.value() );
       if ( msgLevel( MSG::DEBUG ) ) {
         debug() << "Transpose " << thread_id << ": Calculated MEP offsets for slice " << *slice_index
                 << "; good: " << good << "; full: " << transpose_full << "; n_transposed: " << n_transposed << endmsg;
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index f27d62910..962015bb4 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -251,7 +251,7 @@ private:
   std::vector<std::vector<char>> m_event_masks;
 
   // Storage for the currently open input file
-  mutable std::optional<LHCb::StreamDescriptor::Access> m_input = std::nullopt;
+  mutable std::map<std::string, LHCb::StreamDescriptor::Access> m_inputs;
 
   // Iterator that points to the filename of the currently open file
   mutable std::vector<std::string>::const_iterator m_current;
@@ -307,6 +307,7 @@ private:
   Gaudi::Property<std::vector<int>>           m_buffer_numa{this, "BufferNUMA", {}};
   Gaudi::Property<std::vector<std::string>>   m_requests{this, "Requests", {}};
   Gaudi::Property<bool>                       m_thread_per_buffer{this, "ThreadPerBuffer", true};
+  Gaudi::Property<bool>                       m_mask_top5{this, "MaskSourceIDTop5", false};
 
   std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mepsInput;
   std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsInput;
diff --git a/AllenOnline/src/ReadMEP.cpp b/AllenOnline/src/ReadMEP.cpp
index 960da6b04..a4f3acfd8 100644
--- a/AllenOnline/src/ReadMEP.cpp
+++ b/AllenOnline/src/ReadMEP.cpp
@@ -22,11 +22,6 @@
 
 #include <AllenOnline/ReadMEP.h>
 
-namespace {
-  using std::cerr;
-  using std::cout;
-} // namespace
-
 /**
  * @brief      Read a mep from a file
  *
@@ -36,7 +31,8 @@ namespace {
  * @return     (eof, success, mep_header, span of mep data)
  */
 std::tuple<bool, bool, EB::MEP const*, unsigned, gsl::span<char const>>
-MEP::read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer ) {
+MEP::read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer, std::ostream& info_stream,
+               std::ostream& error_stream ) {
   // Allocate space for the first few words of the MEP header
   buffer.resize( sizeof( EB::MEP_header ) );
 
@@ -44,17 +40,17 @@ MEP::read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer
   // Why is the number of bytes returned as an int...
   int read_sc = input.read( &buffer[0], sizeof( EB::MEP_header ) );
   if ( read_sc == 0 ) {
-    cout << "Cannot read more data (Header). End-of-File reached.\n";
+    info_stream << "Cannot read more data (Header). End-of-File reached.\n";
     return {true, false, nullptr, 0u, {}};
   } else if ( read_sc < 0 ) {
-    cerr << "Failed to read header " << strerror( errno ) << "\n";
+    error_stream << "Failed to read header " << strerror( errno ) << "\n";
     return {false, false, nullptr, 0u, {}};
   }
 
   // Check magic pattern
   EB::MEP const* mep = reinterpret_cast<EB::MEP const*>( buffer.data() );
   if ( !mep->is_magic_valid() ) {
-    cerr << "Wrong magic pattern in MEP header: 0x" << std::hex << mep->header.magic << "\n";
+    error_stream << "Wrong magic pattern in MEP header: 0x" << std::hex << mep->header.magic << "\n";
     return {false, false, nullptr, 0u, {}};
   }
 
@@ -71,10 +67,10 @@ MEP::read_mep( LHCb::StreamDescriptor::Access& input, std::vector<char>& buffer
   while ( remaining > 0 ) {
     size_t chunk = std::min( remaining, static_cast<size_t>( std::numeric_limits<int>::max() / 2 ) );
     // Read the next chunk
-    cout << "Reading " << chunk << " bytes\n";
+    info_stream << "Reading " << chunk << " bytes\n";
     read_sc = input.read( pos, static_cast<int>( chunk ) );
     if ( read_sc <= 0 ) {
-      cerr << "Failed to read MEP" << strerror( errno ) << "\n";
+      error_stream << "Failed to read MEP" << strerror( errno ) << "\n";
       return {false, false, nullptr, 0u, {}};
     }
     remaining -= chunk;
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index dfa4a0124..decf85e6c 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -169,7 +169,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
                                                  std::array<unsigned int, NBankTypes> const& mfp_count,
                                                  EventIDs& event_ids, std::vector<char>& event_mask,
                                                  MEP::Blocks const& blocks, std::tuple<size_t, size_t> const& interval,
-                                                 bool split_by_run ) {
+                                                 bool split_by_run, bool mask_top5 ) {
 
   auto [event_start, event_end] = interval;
 
@@ -261,7 +261,7 @@ std::tuple<bool, bool, size_t> MEP::mep_offsets( Allen::Slices& slices, int cons
       std::memcpy( types, bank_types + event_start, ( event_end - event_start ) );
 
       // Store source ID
-      offsets[2 + offset_index] = source_id;
+      offsets[2 + offset_index] = mask_top5 ? source_id & 0x7FF : source_id;
 
       // Initialize the first offsets using the block sizes,
       if ( offset_index < mfp_count[sd_index] - 1 ) {
@@ -292,7 +292,8 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
                          std::unordered_set<BankTypes> const&        subdetectors,
                          std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
                          std::vector<char>& event_mask, EB::MEP const* mep, MEP::Blocks const& blocks,
-                         MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval ) {
+                         MEP::SourceOffsets const& input_offsets, std::tuple<size_t, size_t> const& interval,
+                         bool mask_top5 ) {
   auto [start_event, end_event] = interval;
 
   // Loop over all bank data of this event
@@ -381,7 +382,7 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
 
         // Write sourceID; offset in 32bit words
         auto word_offset         = offset / sizeof( uint32_t );
-        banks_write[word_offset] = source_id;
+        banks_write[word_offset] = mask_top5 ? source_id & 0x7FF : source_id;
 
         // Write bank data
         std::memcpy( banks_write + word_offset + 1, block.payload + source_offsets[i_event], frag_size );
@@ -393,13 +394,12 @@ bool MEP::transpose_mep( Allen::Slices& slices, int const slice_index,
   return true;
 }
 
-std::tuple<bool, bool, size_t> MEP::transpose_events( Allen::Slices& slices, int const slice_index,
-                                                      std::unordered_set<BankTypes> const&        bank_types,
-                                                      std::array<unsigned int, NBankTypes> const& mfp_count,
-                                                      EventIDs& event_ids, std::vector<char>& event_mask,
-                                                      EB::MEP const* mep, MEP::Blocks const& blocks,
-                                                      MEP::SourceOffsets const&         source_offsets,
-                                                      std::tuple<size_t, size_t> const& interval, bool split_by_run ) {
+std::tuple<bool, bool, size_t>
+MEP::transpose_events( Allen::Slices& slices, int const slice_index, std::unordered_set<BankTypes> const& bank_types,
+                       std::array<unsigned int, NBankTypes> const& mfp_count, EventIDs& event_ids,
+                       std::vector<char>& event_mask, EB::MEP const* mep, MEP::Blocks const& blocks,
+                       MEP::SourceOffsets const& source_offsets, std::tuple<size_t, size_t> const& interval,
+                       bool split_by_run, bool mask_top5 ) {
   auto [event_start, event_end] = interval;
 
   bool success = true;
@@ -408,7 +408,7 @@ std::tuple<bool, bool, size_t> MEP::transpose_events( Allen::Slices& slices, int
       allen_offsets( slices, slice_index, bank_types, mfp_count, blocks, source_offsets, interval, split_by_run );
 
   transpose_mep( slices, slice_index, bank_types, mfp_count, event_ids, event_mask, mep, blocks, source_offsets,
-                 {event_start, event_start + to_transpose} );
+                 {event_start, event_start + to_transpose}, mask_top5 );
 
   return {success, to_transpose != ( event_end - event_start ), to_transpose};
 }
diff --git a/AllenOnline/tests/options/allen_output.py b/AllenOnline/tests/options/allen_output.py
new file mode 100644
index 000000000..f5abbdf86
--- /dev/null
+++ b/AllenOnline/tests/options/allen_output.py
@@ -0,0 +1,22 @@
+###############################################################################
+# (c) Copyright 2022 CERN for the benefit of the LHCb Collaboration           #
+#                                                                             #
+# This software is distributed under the terms of the GNU General Public      #
+# Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   #
+#                                                                             #
+# In applying this licence, CERN does not waive the privileges and immunities #
+# granted to it by virtue of its status as an Intergovernmental Organization  #
+# or submit itself to any jurisdiction.                                       #
+###############################################################################
+from Moore import options
+from RecoConf.decoders import default_ft_decoding_version
+
+default_ft_decoding_version.global_bind(value=6)
+
+options.print_freq = 100
+options.data_type = 'Upgrade'
+options.simulation = True
+options.dddb_tag = "dddb-20210617"
+options.conddb_tag = "sim-20210617-vc-md100"
+options.input_type = 'MDF'
+options.input_files = ["mep_passthrough.mdf"]
diff --git a/AllenOnline/tests/options/cpu_decoding.py b/AllenOnline/tests/options/cpu_decoding.py
new file mode 100644
index 000000000..b9eb496a9
--- /dev/null
+++ b/AllenOnline/tests/options/cpu_decoding.py
@@ -0,0 +1,62 @@
+###############################################################################
+# (c) Copyright 2019 CERN for the benefit of the LHCb Collaboration           #
+#                                                                             #
+# This software is distributed under the terms of the GNU General Public      #
+# Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   #
+#                                                                             #
+# In applying this licence, CERN does not waive the privileges and immunities #
+# granted to it by virtue of its status as an Intergovernmental Organization  #
+# or submit itself to any jurisdiction.                                       #
+###############################################################################
+from Moore import options, run_reconstruction
+from Moore.config import Reconstruction, setup_ann_service, get_allen_hlt1_decision_ids
+from RecoConf.rich_reconstruction import make_rich_pixels, default_rich_reco_options
+from RecoConf.calorimeter_reconstruction import make_ecal_digits
+from RecoConf.hlt1_muonid import make_muon_hits
+from PyConf.application import default_raw_event
+from PyConf.Algorithms import HltSelReportsDecoder, HltDecReportsDecoder
+
+from RecoConf.hlt1_tracking import (make_VeloClusterTrackingSIMD_hits,
+                                    make_PrStoreSciFiHits_hits,
+                                    make_PrStorePrUTHits_hits)
+
+
+def make_selreports():
+    return HltSelReportsDecoder(
+        RawEventLocations=default_raw_event(["HltDecReports"],
+                                            options.input_raw_format),
+        SourceID='Hlt1',
+    ).OutputHltSelReportsLocation
+
+
+def make_decreports():
+    return HltDecReportsDecoder(
+        RawEventLocations=default_raw_event(["HltSelReports"],
+                                            options.input_raw_format),
+        SourceID='Hlt1',
+    ).OutputHltDecReportsLocation
+
+
+def decoding():
+
+    setup_ann_service(get_allen_hlt1_decision_ids(), {}, {})
+
+    ecal_raw = default_raw_event(["EcalPacked"])
+
+    decoders = [
+        loc.producer for loc in [
+            make_VeloClusterTrackingSIMD_hits(),
+            make_PrStorePrUTHits_hits(),
+            make_PrStoreSciFiHits_hits(),
+            make_rich_pixels(default_rich_reco_options())['RichDecodedData'],
+            make_ecal_digits(ecal_raw),
+            make_muon_hits(),
+            make_decreports(),
+            make_selreports()
+        ]
+    ]
+
+    return Reconstruction('decoding', decoders, [])
+
+
+run_reconstruction(options, decoding)
diff --git a/AllenOnline/tests/qmtest/decode_allen_output.qmt b/AllenOnline/tests/qmtest/decode_allen_output.qmt
new file mode 100644
index 000000000..8c6433a1c
--- /dev/null
+++ b/AllenOnline/tests/qmtest/decode_allen_output.qmt
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?><!DOCTYPE extension  PUBLIC '-//QM/2.3/Extension//EN'  'http://www.codesourcery.com/qm/dtds/2.3/-//qm/2.3/extension//en.dtd'>
+<!--
+    (c) Copyright 2000-2022 CERN for the benefit of the LHCb Collaboration
+
+    This software is distributed under the terms of the GNU General Public
+    Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".
+
+    In applying this licence, CERN does not waive the privileges and immunities
+    granted to it by virtue of its status as an Intergovernmental Organization
+    or submit itself to any jurisdiction.
+-->
+<!--
+Run CPU decoding on Allen MDF output created from MEP input
+-->
+<extension class="GaudiTest.GaudiExeTest" kind="test">
+  <argument name="program"><text>gaudirun.py</text></argument>
+  <argument name="args"><set>
+    <text>$ALLENONLINEROOT/tests/options/allen_output.py</text>
+    <text>$ALLENONLINEROOT/tests/options/cpu_decoding.py</text>
+  </set></argument>
+  <argument name="prerequisites"><set>
+    <tuple><text>mep_passthrough</text><enumeral>PASS</enumeral></tuple>
+  </set></argument>
+  <argument name="reference"><text>${ALLENONLINEROOT}/tests/refs/decode_allen_output.ref</text></argument>
+  <argument name="use_temp_dir"><enumeral>true</enumeral></argument>
+  <argument name="validator"><text>
+
+from Moore.qmtest.exclusions import remove_known_warnings
+countErrorLines({"FATAL": 0, "ERROR": 0, "WARNING": 2},
+                stdout=remove_known_warnings(stdout))
+
+from Moore.qmtest.exclusions import ref_preprocessor
+validateWithReference(preproc=ref_preprocessor)
+
+  </text></argument>
+</extension>
diff --git a/AllenOnline/tests/qmtest/mep_input.qmt b/AllenOnline/tests/qmtest/mep_input.qmt
new file mode 100644
index 000000000..16bdf4ebc
--- /dev/null
+++ b/AllenOnline/tests/qmtest/mep_input.qmt
@@ -0,0 +1,34 @@
+<?xml version="1.0" ?><!DOCTYPE extension  PUBLIC '-//QM/2.3/Extension//EN'  'http://www.codesourcery.com/qm/dtds/2.3/-//qm/2.3/extension//en.dtd'>
+<!--
+    (c) Copyright 2020 CERN for the benefit of the LHCb Collaboration
+-->
+<!--
+#######################################################
+# SUMMARY OF THIS TEST
+# ...................
+# Author: Roel Aaij
+# Purpose: Use ROOT python bindings to obtain the geometry directly
+#          from the stack and run the Allen event loop
+#######################################################
+-->
+<extension class="GaudiTest.GaudiExeTest" kind="test">
+  <argument name="program"><text>python</text></argument>
+  <argument name="args"><set>
+    <text>${BINARYDUMPERSROOT}/options/allen.py</text>
+    <text>-n</text><text>10</text>
+    <text>--monitoring-filename</text><text>mep_input_hists.root</text>
+    <text>--mep</text><text>${ALLEN_PROJECT_ROOT}/input/minbias/mep/upgrade_mc_minbias_scifi_v5_pf10_retinacluster.mep</text>
+    <text>--mep-mask-source-id-top-5</text>
+  </set></argument>
+  <argument name="reference"><text>${ALLENROOT}/tests/refs/allen_10ev.ref</text></argument>
+  <arguement name="timeout"><integer>600</integer></arguement>
+  <argument name="validator"><text>
+
+from Allen.qmtest.exclusions import preprocessor
+from AllenOnline.qmtest.exclusions import skip_counters
+validateWithReference(preproc=preprocessor, counter_preproc=skip_counters)
+
+countErrorLines({"FATAL":0,"ERROR":0,"WARNING":2})
+
+  </text></argument>
+</extension>
diff --git a/AllenOnline/tests/qmtest/mep_passthrough.qmt b/AllenOnline/tests/qmtest/mep_passthrough.qmt
new file mode 100644
index 000000000..246feba01
--- /dev/null
+++ b/AllenOnline/tests/qmtest/mep_passthrough.qmt
@@ -0,0 +1,41 @@
+<?xml version="1.0" ?><!DOCTYPE extension  PUBLIC '-//QM/2.3/Extension//EN'  'http://www.codesourcery.com/qm/dtds/2.3/-//qm/2.3/extension//en.dtd'>
+<!--
+    (c) Copyright 2020 CERN for the benefit of the LHCb Collaboration
+-->
+<!--
+#######################################################
+# SUMMARY OF THIS TEST
+# ...................
+# Author: Roel Aaij
+# Purpose: Use ROOT python bindings to obtain the geometry directly
+#          from the stack and run the Allen event loop on MEP input
+#######################################################
+-->
+<extension class="GaudiTest.GaudiExeTest" kind="test">
+  <argument name="program"><text>python</text></argument>
+  <argument name="args"><set>
+    <text>${BINARYDUMPERSROOT}/options/allen.py</text>
+    <text>--monitoring-filename</text><text>mep_passthrough.root</text>
+    <text>-m</text><text>750</text>
+    <text>-t</text><text>3</text>
+    <text>-n</text><text>2000</text>
+    <text>--events-per-slice</text><text>500</text>
+    <text>--params</text><text>${ALLEN_PROJECT_ROOT}/input/parameters</text>
+    <text>--tags</text><text>dddb-20210617,sim-20210617-vc-md100</text>
+    <text>--sequence</text><text>${ALLEN_INSTALL_DIR}/constants/hlt1_pp_default.json</text>
+    <text>--mep</text><text>mdf:root://eoslhcb.cern.ch//eos/lhcb/wg/rta/samples/mc/Upgrade_minbias_MD_FTv6_MEP/00146082_00000001_1.mep</text>
+    <text>--mep-mask-source-id-top-5</text>
+    <text>--output-file</text><text>mep_passthrough.mdf</text>
+  </set></argument>
+  <argument name="use_temp_dir"><enumeral>true</enumeral></argument>
+  <argument name="reference"><text>../refs/mep_passthrough.ref</text></argument>
+  <argument name="timeout"><integer>600</integer></argument>
+  <argument name="validator"><text>
+
+from Allen.qmtest.exclusions import preprocessor
+validateWithReference(preproc = preprocessor)
+
+countErrorLines({"FATAL":0,"ERROR":0,"WARNING":2})
+
+  </text></argument>
+</extension>
diff --git a/AllenOnline/tests/qmtest/test_mep_banks.qmt b/AllenOnline/tests/qmtest/test_mep_banks.qmt
new file mode 100644
index 000000000..a8f7a8d08
--- /dev/null
+++ b/AllenOnline/tests/qmtest/test_mep_banks.qmt
@@ -0,0 +1,36 @@
+<?xml version="1.0" ?><!DOCTYPE extension  PUBLIC '-//QM/2.3/Extension//EN'  'http://www.codesourcery.com/qm/dtds/2.3/-//qm/2.3/extension//en.dtd'>
+<!--
+    (c) Copyright 2020 CERN for the benefit of the LHCb Collaboration
+-->
+<!--
+#######################################################
+# SUMMARY OF THIS TEST
+# ...................
+# Author: Roel Aaij
+# Purpose: Use ROOT python bindings to obtain the geometry directly
+#          from the stack and run the Allen event loop on MEP input
+#######################################################
+-->
+<extension class="GaudiTest.GaudiExeTest" kind="test">
+  <argument name="program"><text>allen_mep_test_banks</text></argument>
+  <argument name="args"><set>
+    <text>--nevents</text><text>2000</text>
+    <text>--eps</text><text>500</text>
+    <text>--mep</text><text>mdf:root://eoslhcb.cern.ch//eos/lhcb/wg/rta/samples/mc/Upgrade_minbias_MD_FTv6_MEP/00146082_00000001_1.mep</text>
+    <text>--mdf</text><text>mep_passthrough.mdf</text>
+  </set></argument>
+  <argument name="use_temp_dir"><enumeral>true</enumeral></argument>
+  <argument name="prerequisites"><set>
+    <tuple><text>mep_passthrough</text><enumeral>PASS</enumeral></tuple>
+  </set></argument>
+  <argument name="reference"><text>${ALLENONLINEROOT}/tests/refs/test_mep_banks.ref</text></argument>
+  <argument name="timeout"><integer>600</integer></argument>
+  <argument name="validator"><text>
+
+from AllenOnline.qmtest.exclusions import preprocessor
+validateWithReference(preproc=preprocessor)
+
+countErrorLines({"FATAL" : 0, "ERROR" : 0, "WARNING" : 0})
+
+  </text></argument>
+</extension>
diff --git a/AllenOnline/tests/qmtest/test_mep_banks_transpose.qmt b/AllenOnline/tests/qmtest/test_mep_banks_transpose.qmt
new file mode 100644
index 000000000..c1c197148
--- /dev/null
+++ b/AllenOnline/tests/qmtest/test_mep_banks_transpose.qmt
@@ -0,0 +1,37 @@
+<?xml version="1.0" ?><!DOCTYPE extension  PUBLIC '-//QM/2.3/Extension//EN'  'http://www.codesourcery.com/qm/dtds/2.3/-//qm/2.3/extension//en.dtd'>
+<!--
+    (c) Copyright 2020 CERN for the benefit of the LHCb Collaboration
+-->
+<!--
+#######################################################
+# SUMMARY OF THIS TEST
+# ...................
+# Author: Roel Aaij
+# Purpose: Use ROOT python bindings to obtain the geometry directly
+#          from the stack and run the Allen event loop on MEP input
+#######################################################
+-->
+<extension class="GaudiTest.GaudiExeTest" kind="test">
+  <argument name="program"><text>allen_mep_test_banks</text></argument>
+  <argument name="args"><set>
+    <text>--transpose-mep</text>
+    <text>--nevents</text><text>2000</text>
+    <text>--eps</text><text>500</text>
+    <text>--mep</text><text>mdf:root://eoslhcb.cern.ch//eos/lhcb/wg/rta/samples/mc/Upgrade_minbias_MD_FTv6_MEP/00146082_00000001_1.mep</text>
+    <text>--mdf</text><text>mep_passthrough.mdf</text>
+  </set></argument>
+  <argument name="use_temp_dir"><enumeral>true</enumeral></argument>
+  <argument name="prerequisites"><set>
+    <tuple><text>mep_passthrough</text><enumeral>PASS</enumeral></tuple>
+  </set></argument>
+  <argument name="reference"><text>${ALLENONLINEROOT}/tests/refs/test_mep_banks.ref</text></argument>
+  <argument name="timeout"><integer>600</integer></argument>
+  <argument name="validator"><text>
+
+from AllenOnline.qmtest.exclusions import preprocessor
+validateWithReference(preproc=preprocessor)
+
+countErrorLines({"FATAL" : 0, "ERROR" : 0, "WARNING" : 0})
+
+  </text></argument>
+</extension>
diff --git a/AllenOnline/tests/refs/decode_allen_output.ref b/AllenOnline/tests/refs/decode_allen_output.ref
new file mode 100644
index 000000000..7084fd9dd
--- /dev/null
+++ b/AllenOnline/tests/refs/decode_allen_output.ref
@@ -0,0 +1,9 @@
+ToolSvc                                INFO Removing all tools created by ToolSvc
+ApplicationMgr                         INFO Application Manager Finalized successfully
+ApplicationMgr                         INFO Application Manager Terminated successfully
+FutureEcalZSup                         INFO Number of counters : 1
+ |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
+ |*"No bank found"                                 |      2000 |          0 |(  0.000000 +-  0.000000)% |
+PrStorePrUTHits                        INFO Number of counters : 1
+ |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
+ | "#banks"                                        |      2000 |     432000 |     216.00 |
diff --git a/AllenOnline/tests/refs/mep_passthrough.ref b/AllenOnline/tests/refs/mep_passthrough.ref
new file mode 100644
index 000000000..d3400f960
--- /dev/null
+++ b/AllenOnline/tests/refs/mep_passthrough.ref
@@ -0,0 +1,38 @@
+ApplicationMgr    SUCCESS
+====================================================================================================================================
+====================================================================================================================================
+ApplicationMgr       INFO Application Manager Configured successfully
+DetectorPersist...   INFO Added successfully Conversion service:XmlCnvSvc
+DetectorDataSvc   SUCCESS Detector description database: git:/lhcb.xml
+EventClockSvc.F...   INFO Event times generated from 0 with steps of 0
+DeFTDetector         INFO Current FT geometry version =   64
+DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M2/Cabling
+DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M3/Cabling
+DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M4/Cabling
+DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M5/Cabling
+DumpMuonGeometry     INFO Number of tiles in tell1 table: 19968
+MagneticFieldSvc     INFO Map scaled by factor 1 with polarity internally used: -1 signed relative current: -1
+ToolSvc.PrUTMag...   INFO Start generation of VeloUT Bdl LUTs
+ToolSvc.PrUTMag...   INFO Generation of VeloUT Bdl LUTs finished
+ToolSvc.PrUTMag...   INFO Start generation of VeloUT deflection LUTs
+ToolSvc.PrUTMag...   INFO Generation of VeloUT deflection LUTs finished
+EventLoopMgr      WARNING Unable to locate service "EventSelector"
+EventLoopMgr      WARNING No events will be processed from external input.
+ApplicationMgr       INFO Application Manager Initialized successfully
+ApplicationMgr       INFO Application Manager Started successfully
+Setting number of slices to 4
+Input complete
+Processing complete
+Wrote 2000/2000 events to mep_passthrough.mdf
+ApplicationMgr       INFO Application Manager Stopped successfully
+EventLoopMgr         INFO Histograms converted successfully according to request.
+ToolSvc              INFO Removing all tools created by ToolSvc
+ApplicationMgr       INFO Application Manager Finalized successfully
+ApplicationMgr       INFO Application Manager Terminated successfully
+Bursts               INFO Number of counters : 1
+ |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
+ | "IN"                                            |         1 |
+Events               INFO Number of counters : 2
+ |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
+ | "IN"                                            |      2000 |
+ | "MB_IN"                                         |       679 |
diff --git a/AllenOnline/tests/refs/test_mep_banks.ref b/AllenOnline/tests/refs/test_mep_banks.ref
new file mode 100644
index 000000000..4ce7cc501
--- /dev/null
+++ b/AllenOnline/tests/refs/test_mep_banks.ref
@@ -0,0 +1,17 @@
+mdf_files = mep_passthrough.mdf
+mep_files = mdf:root://eoslhcb.cern.ch//eos/lhcb/wg/rta/samples/mc/Upgrade_minbias_MD_FTv6_MEP/00146082_00000001_1.mep
+ApplicationMgr    SUCCESS
+====================================================================================================================================
+====================================================================================================================================
+ApplicationMgr       INFO Application Manager Configured successfully
+MEPProvider          INFO Providing banks for VP ECal ODIN FT Muon UT
+MEPProvider          INFO Providing events in
+ApplicationMgr       INFO Application Manager Initialized successfully
+MEPProvider          INFO Opened mdf:root://eoslhcb.cern.ch//eos/lhcb/wg/rta/samples/mc/Upgrade_minbias_MD_FTv6_MEP/00146082_00000001_1.mep
+ApplicationMgr       INFO Application Manager Started successfully
+===============================================================================
+All tests passed
+ApplicationMgr       INFO Application Manager Stopped successfully
+EventLoopMgr         INFO Histograms converted successfully according to request.
+ToolSvc              INFO Removing all tools created by ToolSvc
+ApplicationMgr       INFO Application Manager Finalized successfully
-- 
GitLab


From 62ed6225e3b3854651008dbac615d74e7cbf0552 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 9 May 2022 15:09:40 +0200
Subject: [PATCH 095/120] Add OutputBatchSize to MBMOutput

---
 AllenOnline/src/MBMOutput.cpp | 69 +++++++++++++++++++++++++++++-
 AllenOnline/src/MBMOutput.h   | 79 -----------------------------------
 2 files changed, 67 insertions(+), 81 deletions(-)
 delete mode 100644 AllenOnline/src/MBMOutput.h

diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index 3dfba6a76..d687867d3 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -9,11 +9,76 @@
 * or submit itself to any jurisdiction.                                       *
 \*****************************************************************************/
 #include <Gaudi/Accumulators.h>
+#include <GaudiKernel/Service.h>
 
 #include <MDF/MDFHeader.h>
 #include <MDF/RawEventHelpers.h>
 
-#include "MBMOutput.h"
+#include <MBM/bmdef.h>
+#include <RTL/Logger.h>
+
+#include <Allen/OutputHandler.h>
+#include "AllenConfiguration.h"
+
+namespace Allen {
+
+  class MBMOutput final : public Service, public OutputHandler {
+  public:
+    MBMOutput( std::string name, ISvcLocator* loc );
+
+    /// Callback when space is availible (not really used)
+    static int spaceCallback( void* /* param */ );
+
+    StatusCode initialize() override;
+
+    StatusCode start() override;
+
+  protected:
+    std::tuple<size_t, gsl::span<char>> buffer( size_t buffer_size, size_t n_events ) override;
+
+    bool write_buffer( size_t ) override;
+
+    void output_done() override;
+
+    void cancel() override;
+
+  private:
+    SmartIF<AllenConfiguration> m_allenConfig;
+
+    // Output buffer name
+    Gaudi::Property<std::string> m_bufferName{this, "BufferName", "Output"};
+
+    // Do output checksum
+    Gaudi::Property<bool> m_checksum{this, "Checksum", false};
+
+    // Do output checksum
+    Gaudi::Property<size_t> m_outputBatchSize{this, "OutputBatchSize", 10};
+
+    // Process name
+    std::string m_processName;
+
+    // partition ID
+    unsigned m_partitionID = 0;
+
+    // MBM buffer
+    gsl::span<char> m_buffer;
+
+    // Output buffer ID
+    BMID m_bmID = MBM_INV_DESC;
+
+    bool m_cancelled = false;
+
+    std::mutex                         m_mutex;
+    size_t                             m_id = 0;
+    std::list<size_t>                  m_freeIDs;
+    std::unordered_map<size_t, size_t> m_eventCounts;
+
+    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_burstsOutput;
+    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsOutput;
+    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbOutput;
+  };
+
+} // namespace Allen
 
 DECLARE_COMPONENT( Allen::MBMOutput )
 
@@ -105,7 +170,7 @@ StatusCode Allen::MBMOutput::initialize() {
   m_eventsOutput = std::make_unique<Gaudi::Accumulators::Counter<>>( eventsSvc, "OUT" );
   m_mbOutput     = std::make_unique<Gaudi::Accumulators::Counter<>>( eventsSvc, "MB_OUT" );
 
-  init( inputProvider, connection, nLines, m_checksum.value() );
+  init( inputProvider, connection, m_outputBatchSize.value(), nLines, m_checksum.value() );
   return StatusCode::SUCCESS;
 }
 
diff --git a/AllenOnline/src/MBMOutput.h b/AllenOnline/src/MBMOutput.h
deleted file mode 100644
index a920d5249..000000000
--- a/AllenOnline/src/MBMOutput.h
+++ /dev/null
@@ -1,79 +0,0 @@
-/*****************************************************************************\
-* (c) Copyright 2021 CERN for the benefit of the LHCb Collaboration           *
-*                                                                             *
-* This software is distributed under the terms of the GNU General Public      *
-* Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   *
-*                                                                             *
-* In applying this licence, CERN does not waive the privileges and immunities *
-* granted to it by virtue of its status as an Intergovernmental Organization  *
-* or submit itself to any jurisdiction.                                       *
-\*****************************************************************************/
-#pragma once
-
-#include <MBM/bmdef.h>
-#include <RTL/Logger.h>
-
-#include <Allen/OutputHandler.h>
-
-#include <GaudiKernel/Service.h>
-
-#include "AllenConfiguration.h"
-
-namespace Allen {
-
-  class MBMOutput final : public Service, public OutputHandler {
-  public:
-    MBMOutput( std::string name, ISvcLocator* loc );
-
-    // std::unique_ptr<RTL::Logger>&& logger);
-
-    /// Callback when space is availible (not really used)
-    static int spaceCallback( void* /* param */ );
-
-    StatusCode initialize() override;
-
-    StatusCode start() override;
-
-  protected:
-    std::tuple<size_t, gsl::span<char>> buffer( size_t buffer_size, size_t n_events ) override;
-
-    bool write_buffer( size_t ) override;
-
-    void output_done() override;
-
-    void cancel() override;
-
-  private:
-    SmartIF<AllenConfiguration> m_allenConfig;
-
-    // Output buffer name
-    Gaudi::Property<std::string> m_bufferName{this, "BufferName", "Output"};
-
-    // Do output checksum
-    Gaudi::Property<bool> m_checksum{this, "Checksum", false};
-
-    // Process name
-    std::string m_processName;
-
-    // partition ID
-    unsigned m_partitionID = 0;
-
-    // MBM buffer
-    gsl::span<char> m_buffer;
-
-    // Output buffer ID
-    BMID m_bmID = MBM_INV_DESC;
-
-    bool m_cancelled = false;
-
-    std::mutex                         m_mutex;
-    size_t                             m_id = 0;
-    std::list<size_t>                  m_freeIDs;
-    std::unordered_map<size_t, size_t> m_eventCounts;
-
-    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_burstsOutput;
-    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_eventsOutput;
-    std::unique_ptr<Gaudi::Accumulators::Counter<>> m_mbOutput;
-  };
-
-} // namespace Allen
-- 
GitLab


From 41c62b21982a8662d58ea0a1766e04c9f078da10 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 12 May 2022 11:32:33 +0200
Subject: [PATCH 096/120] Remove obsolete property and allow output batch size
 to be configured

---
 AllenOnline/options/AllenConfig.py   | 5 +++--
 AllenOnline/src/AllenConfiguration.h | 1 -
 AllenOnline/src/MBMOutput.cpp        | 4 ++--
 3 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 6478b7db7..8a9137a92 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -92,7 +92,6 @@ allen_conf.NThreads = 8
 allen_conf.Device = "0"
 allen_conf.JSON = json_file
 allen_conf.NLines = n_lines
-allen_conf.OutputChecksum = False
 allen_conf.ParamDir = "/group/hlt/fest_202106/dev-dir/Allen/input/parameters"
 allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
@@ -103,8 +102,10 @@ if run_online:
     from Configurables import Allen__MBMOutput as MBMOutput
     output_svc = MBMOutput("MBMOutput")
     output_svc.OutputLevel = 2
-    output_svc.Checksum = True
+    # FIXME: checksums disabled until performance is improved
+    output_svc.Checksum = False
     output_svc.BufferName = mbm_setup.Allen_Output
+    output_svc.OutputBatchSize = 100
     appMgr.ExtSvc += [output_svc]
     allen_conf.Output = "mbm://" + output_svc.getFullName()
 elif integration_test:
diff --git a/AllenOnline/src/AllenConfiguration.h b/AllenOnline/src/AllenConfiguration.h
index a3b04bd40..6565c4b28 100644
--- a/AllenOnline/src/AllenConfiguration.h
+++ b/AllenOnline/src/AllenConfiguration.h
@@ -70,7 +70,6 @@ public:
         }
         paramDir.set( dir );
       }};
-  Gaudi::Property<bool>        checksum{this, "OutputChecksum", false};
   Gaudi::Property<unsigned>    partitionID{this, "PartitionID", 0};
   Gaudi::Property<bool>        partitionBuffers{this, "PartitionBuffers", true};
   Gaudi::Property<std::string> partition{this, "Partition", ""};
diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
index d687867d3..9c668c112 100644
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -17,8 +17,8 @@
 #include <MBM/bmdef.h>
 #include <RTL/Logger.h>
 
-#include <Allen/OutputHandler.h>
 #include "AllenConfiguration.h"
+#include <Allen/OutputHandler.h>
 
 namespace Allen {
 
@@ -52,7 +52,7 @@ namespace Allen {
     Gaudi::Property<bool> m_checksum{this, "Checksum", false};
 
     // Do output checksum
-    Gaudi::Property<size_t> m_outputBatchSize{this, "OutputBatchSize", 10};
+    Gaudi::Property<size_t> m_outputBatchSize{this, "OutputBatchSize", 100};
 
     // Process name
     std::string m_processName;
-- 
GitLab


From 40954fcf39b8f524fc966b26b55b1e692bb7481c Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Fri, 13 May 2022 15:31:44 +0200
Subject: [PATCH 097/120] Add missing files

---
 .../python/AllenOnline/qmtest/exclusions.py   | 27 +++++++++++++++++++
 1 file changed, 27 insertions(+)
 create mode 100644 AllenOnline/python/AllenOnline/qmtest/exclusions.py

diff --git a/AllenOnline/python/AllenOnline/qmtest/exclusions.py b/AllenOnline/python/AllenOnline/qmtest/exclusions.py
new file mode 100644
index 000000000..006dc498b
--- /dev/null
+++ b/AllenOnline/python/AllenOnline/qmtest/exclusions.py
@@ -0,0 +1,27 @@
+###############################################################################
+# (c) Copyright 2019 CERN for the benefit of the LHCb Collaboration           #
+#                                                                             #
+# This software is distributed under the terms of the GNU General Public      #
+# Licence version 3 (GPL Version 3), copied verbatim in the file "COPYING".   #
+#                                                                             #
+# In applying this licence, CERN does not waive the privileges and immunities #
+# granted to it by virtue of its status as an Intergovernmental Organization  #
+# or submit itself to any jurisdiction.                                       #
+###############################################################################
+from GaudiConf.QMTest.LHCbTest import BlockSkipper
+from GaudiTesting.BaseTest import RegexpReplacer, LineSkipper
+from GaudiConf.QMTest.LHCbExclusions import preprocessor as LHCbPreprocessor
+
+skip_counters = BlockSkipper("Bursts",
+                             "|        10 |")
+
+skip_start = BlockSkipper("mdf_files = mep_passthrough.mdf"
+                          "Application Manager Configured successfully")
+
+assertion_counter = (RegexpReplacer(" \(\d+ assertions in \d+ test cases\)") +
+                     RegexpReplacer("(Providing events in).*", r"\1"))
+
+skipper = LineSkipper(regexps=["Opened mep_passthrough.mdf",
+                               "Cannot read more data.*"])
+
+preprocessor = (skip_start + skipper + LHCbPreprocessor + assertion_counter)
-- 
GitLab


From 1b43d10e85bfe88cf7e8c5222f0464f03785b46f Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Fri, 13 May 2022 15:57:13 +0200
Subject: [PATCH 098/120] make compile with MPI in stack setup

---
 AllenOnline/CMakeLists.txt           | 20 ++++++++++----------
 AllenOnline/src/AllenApplication.cpp |  1 -
 2 files changed, 10 insertions(+), 11 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 4ece9f850..7c511c90d 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -70,19 +70,19 @@ gaudi_add_executable(allen_test_host_register
       Boost::program_options
 )
 
-gaudi_add_executable(allen_mpi_send
-   SOURCES
-     application/MPISend.cpp
-   LINK
-     AllenOnlineLib
-     LHCb::MDFLib
-     Boost::program_options
-)
-
 # MPI
-if (TARGET MPI::MPI_CXX AND TARGET PkgConfig::hwloc)
+if (TARGET MPI::MPI_CXX AND MPI_CXX_COMPILER AND TARGET PkgConfig::hwloc)
   message(STATUS "Found MPI ${MPI_CXX_VERSION}: ${MPI_CXX_COMPILER}")
 
+    gaudi_add_executable(allen_mpi_send
+    SOURCES
+	 application/MPISend.cpp
+     LINK
+	AllenOnlineLib
+     	LHCb::MDFLib
+     	Boost::program_options
+     )
+
   # open-pal library is also needed...
   set(MPI_LIBDIRS)
   foreach(mpi_lib ${MPI_CXX_LIBRARIES})
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index cc2c725d2..502b9a72e 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -48,7 +48,6 @@
 
 #include "AllenApplication.h"
 #include "AllenConfiguration.h"
-#include "MBMOutput.h"
 
 // #include "EBProvider.h"
 
-- 
GitLab


From 00597f12159b5c83f1a86a91a79ecb2f4236551c Mon Sep 17 00:00:00 2001
From: Gitlab CI <noreply@cern.ch>
Date: Fri, 13 May 2022 13:58:06 +0000
Subject: [PATCH 099/120] Fixed formatting

patch generated by https://gitlab.cern.ch/lhcb/MooreOnline/-/jobs/21776457
---
 AllenOnline/python/AllenOnline/qmtest/exclusions.py | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/AllenOnline/python/AllenOnline/qmtest/exclusions.py b/AllenOnline/python/AllenOnline/qmtest/exclusions.py
index 006dc498b..36f00c1ff 100644
--- a/AllenOnline/python/AllenOnline/qmtest/exclusions.py
+++ b/AllenOnline/python/AllenOnline/qmtest/exclusions.py
@@ -12,8 +12,7 @@ from GaudiConf.QMTest.LHCbTest import BlockSkipper
 from GaudiTesting.BaseTest import RegexpReplacer, LineSkipper
 from GaudiConf.QMTest.LHCbExclusions import preprocessor as LHCbPreprocessor
 
-skip_counters = BlockSkipper("Bursts",
-                             "|        10 |")
+skip_counters = BlockSkipper("Bursts", "|        10 |")
 
 skip_start = BlockSkipper("mdf_files = mep_passthrough.mdf"
                           "Application Manager Configured successfully")
@@ -21,7 +20,7 @@ skip_start = BlockSkipper("mdf_files = mep_passthrough.mdf"
 assertion_counter = (RegexpReplacer(" \(\d+ assertions in \d+ test cases\)") +
                      RegexpReplacer("(Providing events in).*", r"\1"))
 
-skipper = LineSkipper(regexps=["Opened mep_passthrough.mdf",
-                               "Cannot read more data.*"])
+skipper = LineSkipper(
+    regexps=["Opened mep_passthrough.mdf", "Cannot read more data.*"])
 
 preprocessor = (skip_start + skipper + LHCbPreprocessor + assertion_counter)
-- 
GitLab


From 418dee0f6bf790b0e095f7f4965b453df83b6820 Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Wed, 18 May 2022 16:09:15 +0200
Subject: [PATCH 100/120] get sequence from trigger configuration panel

---
 AllenOnline/options/AllenConfig.py | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 8a9137a92..83d023bba 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -31,6 +31,7 @@ try:
     output_level = OnlineEnv.OutputLevel
     partition = OnlineEnv.PartitionName
     partition_id = OnlineEnv.PartitionID
+    allen_sequence = OnlineEnv.HLTType
 except ImportError:
     run_online = False
     output_level = 3
@@ -58,9 +59,7 @@ CondDB().Upgrade = True
 # Get the number of lines from the configuration. This should probably
 # be done internally to Allen and retrieved somehow
 n_lines = 0
-# json_file = "/daqarea1/fest/allen/configuration/passthrough.json"
-json_file = os.path.expandvars(
-    "${ALLEN_INSTALL_DIR}/constants/passthrough.json")
+json_file = os.path.expandvars("${ALLEN_INSTALL_DIR}/constants/") + allen_sequence + ".json"
 with open(json_file) as jf:
     json_config = json.load(jf)
     gs = json_config.get('gather_selections', {})
@@ -92,7 +91,7 @@ allen_conf.NThreads = 8
 allen_conf.Device = "0"
 allen_conf.JSON = json_file
 allen_conf.NLines = n_lines
-allen_conf.ParamDir = "/group/hlt/fest_202106/dev-dir/Allen/input/parameters"
+allen_conf.ParamDir = os.getenv("PARAMFILESROOT")
 allen_conf.Partition = partition
 allen_conf.PartitionBuffers = True
 allen_conf.PartitionID = partition_id
-- 
GitLab


From 3199b5e34474fc402ce9aa5267cd212adaba8568 Mon Sep 17 00:00:00 2001
From: Gitlab CI <noreply@cern.ch>
Date: Wed, 18 May 2022 14:09:38 +0000
Subject: [PATCH 101/120] Fixed formatting

patch generated by https://gitlab.cern.ch/lhcb/MooreOnline/-/jobs/21898673
---
 AllenOnline/options/AllenConfig.py | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 83d023bba..8bf356169 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -59,7 +59,8 @@ CondDB().Upgrade = True
 # Get the number of lines from the configuration. This should probably
 # be done internally to Allen and retrieved somehow
 n_lines = 0
-json_file = os.path.expandvars("${ALLEN_INSTALL_DIR}/constants/") + allen_sequence + ".json"
+json_file = os.path.expandvars(
+    "${ALLEN_INSTALL_DIR}/constants/") + allen_sequence + ".json"
 with open(json_file) as jf:
     json_config = json.load(jf)
     gs = json_config.get('gather_selections', {})
-- 
GitLab


From a89659bf20c48d507ddbeb2e66083f4e3255049b Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Fri, 20 May 2022 16:28:59 +0200
Subject: [PATCH 102/120] Take DB tags from OnlineEnv

---
 AllenOnline/options/AllenConfig.py | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 8bf356169..6a2b59926 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -32,6 +32,8 @@ try:
     partition = OnlineEnv.PartitionName
     partition_id = OnlineEnv.PartitionID
     allen_sequence = OnlineEnv.HLTType
+    dddb_tag = OnlineEnv.DDDBTag
+    conddb_tag = OnlineEnv.CondDBTag
 except ImportError:
     run_online = False
     output_level = 3
@@ -43,8 +45,8 @@ integration_test = False
 app = LHCbApp(
     DataType="Upgrade",
     Simulation=True,
-    DDDBtag="dddb-20210617",
-    CondDBtag="sim-20210617-vc-md100")
+    DDDBtag=dddb_tag,
+    CondDBtag=conddb_tag)
 # DDDBtag="dddb-20210218",
 # CondDBtag="sim-20201218-vc-md100")
 
-- 
GitLab


From 31bdd953d13225e8d09e9d9206e495009a83e7bc Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Fri, 20 May 2022 16:33:41 +0200
Subject: [PATCH 103/120] Fix build of (empty) MPISend when MPI is not found

---
 AllenOnline/CMakeLists.txt                  | 17 +++++++++--------
 AllenOnline/include/AllenOnline/MPIConfig.h |  2 ++
 2 files changed, 11 insertions(+), 8 deletions(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 7c511c90d..50c7c6a6f 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -70,18 +70,19 @@ gaudi_add_executable(allen_test_host_register
       Boost::program_options
 )
 
+gaudi_add_executable(allen_mpi_send
+    SOURCES
+      application/MPISend.cpp
+    LINK
+      AllenOnlineLib
+      LHCb::MDFLib
+      Boost::program_options
+)
+
 # MPI
 if (TARGET MPI::MPI_CXX AND MPI_CXX_COMPILER AND TARGET PkgConfig::hwloc)
   message(STATUS "Found MPI ${MPI_CXX_VERSION}: ${MPI_CXX_COMPILER}")
 
-    gaudi_add_executable(allen_mpi_send
-    SOURCES
-	 application/MPISend.cpp
-     LINK
-	AllenOnlineLib
-     	LHCb::MDFLib
-     	Boost::program_options
-     )
 
   # open-pal library is also needed...
   set(MPI_LIBDIRS)
diff --git a/AllenOnline/include/AllenOnline/MPIConfig.h b/AllenOnline/include/AllenOnline/MPIConfig.h
index 4fbe5ef3a..e11ed97a4 100644
--- a/AllenOnline/include/AllenOnline/MPIConfig.h
+++ b/AllenOnline/include/AllenOnline/MPIConfig.h
@@ -4,7 +4,9 @@
 #pragma once
 
 #include <limits.h>
+#ifdef HAVE_MPI
 #include <mpi.h>
+#endif
 
 // Determine size of size_t for MPI type
 #if SIZE_MAX == UCHAR_MAX
-- 
GitLab


From 378939b5ab45eeae7f4122e13d8cc8bb5a83d3a5 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 24 May 2022 15:57:17 +0200
Subject: [PATCH 104/120] Follow rebase of propage_bank_sizes in Allen

---
 AllenOnline/include/AllenOnline/MPIConfig.h   |  1 +
 .../include/AllenOnline/TransposeMEP.h        |  4 ++++
 AllenOnline/src/AllenApplication.cpp          |  8 +++++--
 AllenOnline/src/AllenApplication.h            |  2 +-
 AllenOnline/src/MEPProvider.cpp               | 19 ++++++++++++----
 AllenOnline/src/MEPProvider.h                 |  3 ++-
 AllenOnline/src/TransposeMEP.cpp              | 22 +++++++++----------
 7 files changed, 40 insertions(+), 19 deletions(-)

diff --git a/AllenOnline/include/AllenOnline/MPIConfig.h b/AllenOnline/include/AllenOnline/MPIConfig.h
index e11ed97a4..163f93b7b 100644
--- a/AllenOnline/include/AllenOnline/MPIConfig.h
+++ b/AllenOnline/include/AllenOnline/MPIConfig.h
@@ -4,6 +4,7 @@
 #pragma once
 
 #include <limits.h>
+
 #ifdef HAVE_MPI
 #include <mpi.h>
 #endif
diff --git a/AllenOnline/include/AllenOnline/TransposeMEP.h b/AllenOnline/include/AllenOnline/TransposeMEP.h
index 0bba72116..2cca5a8f1 100644
--- a/AllenOnline/include/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/include/AllenOnline/TransposeMEP.h
@@ -64,6 +64,10 @@ namespace MEP {
   };
   using Slices = std::vector<MEP::Slice>;
 
+  LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes,
+                          const uint8_t version );
+
+
   /**
    * @brief      Fill the array the contains the number of banks per type
    *
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 502b9a72e..69ce1d956 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -289,7 +289,11 @@ void AllenApplication::allenLoop() {
   allen( m_options, this, m_provider, m_output, m_zmqSvc.get(), m_controlConnection );
 }
 
-void AllenApplication::update( unsigned long run ) {
+void AllenApplication::update(gsl::span<unsigned const> odin_data)
+{
+  LHCb::ODIN odin{odin_data};
+  auto const run = odin.runNumber();
+
   // Monitoring run change
   if ( m_runNumber == 0 ) {
     m_runNumber = run;
@@ -302,7 +306,7 @@ void AllenApplication::update( unsigned long run ) {
   }
 
   // Update conditions by forwarding to the real updater
-  m_updater->update( run );
+  m_updater->update( odin_data );
 }
 
 void AllenApplication::registerConsumer( std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c ) {
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index 2fd15a647..b2123e461 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -66,7 +66,7 @@ public:
   bool initMPI();
 
   // Updater
-  void update( unsigned long run ) override;
+  void update(gsl::span<unsigned const> odin_data) override;
 
   void registerConsumer( std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c ) override;
 
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index 59d5d5d05..abb345dc6 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -9,6 +9,7 @@
 #include <Allen/Provider.h>
 #include <Allen/SliceUtils.h>
 #include <Allen/Timer.h>
+#include <Allen/MEPTools.h>
 #include <Allen/sourceid.h>
 #include <Allen/write_mdf.hpp>
 #include <Backend/BackendCommon.h>
@@ -86,10 +87,10 @@ BanksAndOffsets MEPProvider::banks( BankTypes bank_type, size_t slice_index ) co
   return bno;
 }
 
-std::tuple<bool, bool, bool, size_t, size_t, uint> MEPProvider::get_slice( std::optional<unsigned int> timeout ) {
+std::tuple<bool, bool, bool, size_t, size_t, std::any> MEPProvider::get_slice( std::optional<unsigned int> timeout ) {
   bool                         timed_out = false, done = false;
   size_t                       slice_index = 0, n_filled = 0;
-  uint                         run_no = 0;
+  std::any                     odin_span;
   std::unique_lock<std::mutex> lock{m_transpose_mut};
 
   if ( !m_read_error ) {
@@ -111,7 +112,17 @@ std::tuple<bool, bool, bool, size_t, size_t, uint> MEPProvider::get_slice( std::
     if ( !m_read_error && !m_transposed.empty() && ( !timeout || ( timeout && !timed_out ) ) ) {
       std::tie( slice_index, n_filled ) = m_transposed.front();
       m_transposed.pop_front();
-      if ( n_filled > 0 ) { run_no = std::get<0>( m_event_ids[slice_index].front() ); }
+      if ( n_filled > 0 ) {
+        auto bno = banks(BankTypes::ODIN, slice_index);
+        auto const* block = bno.fragments[0].data();
+        auto const* offsets = bno.offsets.data();
+        auto const n_banks = offsets[0];
+        auto odin = MEP::decode_odin( block,
+                                      offsets[MEP::offset_index(n_banks, 0, 0)],
+                                      MEP::bank_size(block, bno.sizes.data(), 0, 0),
+                                      bno.version );
+        odin_span = gsl::span<unsigned const>{odin.data};
+      }
     }
   }
 
@@ -126,7 +137,7 @@ std::tuple<bool, bool, bool, size_t, size_t, uint> MEPProvider::get_slice( std::
             << n_filled << endmsg;
   }
 
-  return {!m_read_error, done, timed_out, slice_index, m_read_error ? 0 : n_filled, run_no};
+  return {!m_read_error, done, timed_out, slice_index, m_read_error ? 0 : n_filled, odin_span};
 }
 
 void MEPProvider::slice_free( size_t slice_index ) {
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index 962015bb4..7f1de6c5e 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -14,6 +14,7 @@
 #include <numeric>
 #include <thread>
 #include <vector>
+#include <any>
 
 #include <fcntl.h>
 #include <sys/stat.h>
@@ -111,7 +112,7 @@ public:
    *
    * @return     (good slice, timed out, slice index, number of events in slice)
    */
-  std::tuple<bool, bool, bool, size_t, size_t, uint>
+  std::tuple<bool, bool, bool, size_t, size_t, std::any>
   get_slice( std::optional<unsigned int> timeout = std::nullopt ) override;
 
   /**
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index decf85e6c..bd33f582e 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -10,17 +10,6 @@
 #include <AllenOnline/TransposeMEP.h>
 
 namespace {
-  LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes,
-                          const uint8_t version ) {
-    LHCb::span<std::uint32_t const> odin_span{reinterpret_cast<std::uint32_t const*>( odin_data + offset ),
-                                              size_bytes / sizeof( uint32_t )};
-    if ( version == 6 ) {
-      return LHCb::ODIN::from_version<6>( odin_span );
-    } else {
-      return LHCb::ODIN{odin_span};
-    }
-  }
-
   BankTypes source_id_type( uint16_t src_id ) {
     auto const sd = SourceId_sys( src_id );
     auto       it = Allen::subdetectors.find( static_cast<SourceIdSys>( sd ) );
@@ -52,6 +41,17 @@ MEP::fill_counts( EB::MEP const* mep ) {
   return {true, count, versions};
 }
 
+LHCb::ODIN MEP::decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes,
+                             const uint8_t version ) {
+  LHCb::span<std::uint32_t const> odin_span{reinterpret_cast<std::uint32_t const*>( odin_data + offset ),
+    size_bytes / sizeof( uint32_t )};
+  if ( version == 6 ) {
+    return LHCb::ODIN::from_version<6>( odin_span );
+  } else {
+    return LHCb::ODIN{odin_span};
+  }
+}
+
 void MEP::find_blocks( EB::MEP const* mep, Blocks& blocks ) {
   // Fill blocks in temporary container
   Blocks tmp{blocks};
-- 
GitLab


From b6fb4f2a422c0a73920c5d0264c3404dc40c05f2 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 24 May 2022 21:55:07 +0200
Subject: [PATCH 105/120] Fix most tests

---
 AllenOnline/src/MEPProvider.cpp               |  4 +-
 AllenOnline/tests/qmtest/mep_passthrough.qmt  |  3 +-
 .../tests/refs/decode_allen_output.ref        | 27 +++++++
 AllenOnline/tests/refs/mep_passthrough.ref    | 73 +++++++++++--------
 AllenOnline/tests/refs/test_mep_banks.ref     |  2 +-
 5 files changed, 76 insertions(+), 33 deletions(-)

diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index abb345dc6..faccaf072 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -271,7 +271,9 @@ StatusCode MEPProvider::initialize() {
     return StatusCode::FAILURE;
   } else {
     info() << "Providing banks for";
-    for ( auto bt : m_bank_types ) info() << " " << ::bank_name( bt );
+    std::vector<BankTypes> types{m_bank_types.begin(), m_bank_types.end()};
+    std::sort(types.begin(), types.end());
+    for ( auto bt : types ) info() << " " << ::bank_name( bt );
     info() << endmsg;
   }
 
diff --git a/AllenOnline/tests/qmtest/mep_passthrough.qmt b/AllenOnline/tests/qmtest/mep_passthrough.qmt
index 246feba01..707bbc0eb 100644
--- a/AllenOnline/tests/qmtest/mep_passthrough.qmt
+++ b/AllenOnline/tests/qmtest/mep_passthrough.qmt
@@ -20,7 +20,6 @@
     <text>-t</text><text>3</text>
     <text>-n</text><text>2000</text>
     <text>--events-per-slice</text><text>500</text>
-    <text>--params</text><text>${ALLEN_PROJECT_ROOT}/input/parameters</text>
     <text>--tags</text><text>dddb-20210617,sim-20210617-vc-md100</text>
     <text>--sequence</text><text>${ALLEN_INSTALL_DIR}/constants/hlt1_pp_default.json</text>
     <text>--mep</text><text>mdf:root://eoslhcb.cern.ch//eos/lhcb/wg/rta/samples/mc/Upgrade_minbias_MD_FTv6_MEP/00146082_00000001_1.mep</text>
@@ -35,7 +34,7 @@
 from Allen.qmtest.exclusions import preprocessor
 validateWithReference(preproc = preprocessor)
 
-countErrorLines({"FATAL":0,"ERROR":0,"WARNING":2})
+countErrorLines({"FATAL":0,"ERROR":0,"WARNING":0})
 
   </text></argument>
 </extension>
diff --git a/AllenOnline/tests/refs/decode_allen_output.ref b/AllenOnline/tests/refs/decode_allen_output.ref
index 7084fd9dd..bda29802a 100644
--- a/AllenOnline/tests/refs/decode_allen_output.ref
+++ b/AllenOnline/tests/refs/decode_allen_output.ref
@@ -7,3 +7,30 @@ FutureEcalZSup                         INFO Number of counters : 1
 PrStorePrUTHits                        INFO Number of counters : 1
  |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
  | "#banks"                                        |      2000 |     432000 |     216.00 |
+PrStoreSciFiHits                       INFO Number of counters : 25
+ |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
+ | "Average X in T1U"                              |    560385 |-2.482986e+07 |    -44.309 |     1147.8 |     -2656.4 |      2656.3 |
+ | "Average X in T1V"                              |    564959 |-1.886819e+07 |    -33.397 |     1135.9 |     -2656.4 |      2656.3 |
+ | "Average X in T1X1"                             |    554626 |-3.213596e+07 |    -57.942 |     1170.1 |     -2646.2 |      2646.2 |
+ | "Average X in T1X2"                             |    572727 |-1.189806e+07 |    -20.774 |     1124.4 |     -2646.2 |      2646.2 |
+ | "Average X in T2U"                              |    542815 |-1.062755e+07 |    -19.579 |     1141.5 |     -2656.4 |      2656.3 |
+ | "Average X in T2V"                              |    559143 |-1.210081e+07 |    -21.642 |     1136.1 |     -2656.4 |      2656.3 |
+ | "Average X in T2X1"                             |    517680 |-1.140797e+07 |    -22.037 |     1144.4 |     -2646.2 |      2646.2 |
+ | "Average X in T2X2"                             |    577655 |   -7098556 |    -12.289 |     1130.1 |     -2646.2 |      2646.2 |
+ | "Average X in T3U"                              |    588838 |   -7559260 |    -12.838 |     1341.6 |     -3188.4 |      3188.4 |
+ | "Average X in T3V"                              |    608287 |   -8528432 |    -14.020 |     1333.9 |     -3188.4 |      3188.4 |
+ | "Average X in T3X1"                             |    566054 |   -4487447 |    -7.9276 |     1342.2 |     -3176.2 |      3176.2 |
+ | "Average X in T3X2"                             |    630129 |-1.034146e+07 |    -16.412 |     1327.0 |     -3176.2 |      3176.1 |
+ | "Hits in T1U"                                   |      8000 |     560385 |     70.048 |     37.658 |       0.0000 |      291.00 |
+ | "Hits in T1V"                                   |      8000 |     564959 |     70.620 |     38.085 |       0.0000 |      302.00 |
+ | "Hits in T1X1"                                  |      8000 |     554626 |     69.328 |     37.165 |       0.0000 |      317.00 |
+ | "Hits in T1X2"                                  |      8000 |     572727 |     71.591 |     38.529 |       0.0000 |      299.00 |
+ | "Hits in T2U"                                   |      8000 |     542815 |     67.852 |     37.138 |      1.0000 |      347.00 |
+ | "Hits in T2V"                                   |      8000 |     559143 |     69.893 |     38.452 |      1.0000 |      310.00 |
+ | "Hits in T2X1"                                  |      8000 |     517680 |     64.710 |     35.041 |      1.0000 |      288.00 |
+ | "Hits in T2X2"                                  |      8000 |     577655 |     72.207 |     39.142 |      1.0000 |      299.00 |
+ | "Hits in T3U"                                   |      8000 |     588838 |     73.605 |     39.458 |      1.0000 |      352.00 |
+ | "Hits in T3V"                                   |      8000 |     608287 |     76.036 |     40.836 |      1.0000 |      320.00 |
+ | "Hits in T3X1"                                  |      8000 |     566054 |     70.757 |     37.656 |      2.0000 |      304.00 |
+ | "Hits in T3X2"                                  |      8000 |     630129 |     78.766 |     42.000 |      1.0000 |      329.00 |
+ | "Total number of hits"                          |      2000 |    6843298 |     3421.6 |     1714.1 |      147.00 |      11692.0 |
diff --git a/AllenOnline/tests/refs/mep_passthrough.ref b/AllenOnline/tests/refs/mep_passthrough.ref
index d3400f960..ccc6f3123 100644
--- a/AllenOnline/tests/refs/mep_passthrough.ref
+++ b/AllenOnline/tests/refs/mep_passthrough.ref
@@ -1,38 +1,53 @@
-ApplicationMgr    SUCCESS
-====================================================================================================================================
-====================================================================================================================================
-ApplicationMgr       INFO Application Manager Configured successfully
-DetectorPersist...   INFO Added successfully Conversion service:XmlCnvSvc
-DetectorDataSvc   SUCCESS Detector description database: git:/lhcb.xml
-EventClockSvc.F...   INFO Event times generated from 0 with steps of 0
-DeFTDetector         INFO Current FT geometry version =   64
-DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M2/Cabling
-DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M3/Cabling
-DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M4/Cabling
-DumpMuonGeometry     INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M5/Cabling
-DumpMuonGeometry     INFO Number of tiles in tell1 table: 19968
-MagneticFieldSvc     INFO Map scaled by factor 1 with polarity internally used: -1 signed relative current: -1
-ToolSvc.PrUTMag...   INFO Start generation of VeloUT Bdl LUTs
-ToolSvc.PrUTMag...   INFO Generation of VeloUT Bdl LUTs finished
-ToolSvc.PrUTMag...   INFO Start generation of VeloUT deflection LUTs
-ToolSvc.PrUTMag...   INFO Generation of VeloUT deflection LUTs finished
-EventLoopMgr      WARNING Unable to locate service "EventSelector"
-EventLoopMgr      WARNING No events will be processed from external input.
-ApplicationMgr       INFO Application Manager Initialized successfully
-ApplicationMgr       INFO Application Manager Started successfully
+DetectorPersistencySvc                 INFO Added successfully Conversion service:XmlCnvSvc
+DetectorDataSvc                     SUCCESS Detector description database: git:/lhcb.xml
+EventClockSvc.FakeEventTime            INFO Event times generated from 0 with steps of 0
+MagneticFieldSvc                       INFO Map scaled by factor 1 with polarity internally used: -1 signed relative current: -1
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M2/Cabling
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M3/Cabling
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M4/Cabling
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M5/Cabling
+DumpMuonGeometry                       INFO Number of tiles in tell1 table: 19968
+ToolSvc.PrUTMagnetTool                 INFO Start generation of VeloUT Bdl LUTs
+ToolSvc.PrUTMagnetTool                 INFO Generation of VeloUT Bdl LUTs finished
+ToolSvc.PrUTMagnetTool                 INFO Start generation of VeloUT deflection LUTs
+ToolSvc.PrUTMagnetTool                 INFO Generation of VeloUT deflection LUTs finished
+HLTControlFlowMgr                      INFO Start initialization
+HLTControlFlowMgr                      INFO Will not use an EventSelector.
+HLTControlFlowMgr                      INFO Concurrency level information:
+HLTControlFlowMgr                      INFO  o Number of events slots: 1
+HLTControlFlowMgr                      INFO  o TBB thread pool size:  'ThreadPoolSize':1
+ApplicationMgr                         INFO Application Manager Initialized successfully
+DeFTDetector                           INFO Current FT geometry version =   64
+ApplicationMgr                         INFO Application Manager Started successfully
 Setting number of slices to 4
+DumpFTGeometry                         INFO Conditions DB is compatible with FT bank version 4, 5, 6.
 Input complete
 Processing complete
 Wrote 2000/2000 events to mep_passthrough.mdf
-ApplicationMgr       INFO Application Manager Stopped successfully
-EventLoopMgr         INFO Histograms converted successfully according to request.
-ToolSvc              INFO Removing all tools created by ToolSvc
-ApplicationMgr       INFO Application Manager Finalized successfully
-ApplicationMgr       INFO Application Manager Terminated successfully
-Bursts               INFO Number of counters : 1
+ApplicationMgr                         INFO Application Manager Stopped successfully
+HLTControlFlowMgr                      INFO
+HLTControlFlowMgr                      INFO
+LAZY_AND: allen_non_event_data                #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+ NONLAZY_OR: allen_non_event_data_converters  #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpBeamline/DumpBeamline                   #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpCaloGeometry/DumpCaloGeometry           #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpVPGeometry/DumpVPGeometry               #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpMagneticField/DumpMagneticField         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpFTGeometry/DumpFTGeometry               #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+ NONLAZY_OR: allen_non_event_data_producers   #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer           #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#1         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#2         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#3         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#4         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+HLTControlFlowMgr                      INFO Histograms converted successfully according to request.
+ToolSvc                                INFO Removing all tools created by ToolSvc
+ApplicationMgr                         INFO Application Manager Finalized successfully
+ApplicationMgr                         INFO Application Manager Terminated successfully
+Bursts                                 INFO Number of counters : 1
  |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
  | "IN"                                            |         1 |
-Events               INFO Number of counters : 2
+Events                                 INFO Number of counters : 2
  |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
  | "IN"                                            |      2000 |
  | "MB_IN"                                         |       679 |
diff --git a/AllenOnline/tests/refs/test_mep_banks.ref b/AllenOnline/tests/refs/test_mep_banks.ref
index 4ce7cc501..83bf0366f 100644
--- a/AllenOnline/tests/refs/test_mep_banks.ref
+++ b/AllenOnline/tests/refs/test_mep_banks.ref
@@ -4,7 +4,7 @@ ApplicationMgr    SUCCESS
 ====================================================================================================================================
 ====================================================================================================================================
 ApplicationMgr       INFO Application Manager Configured successfully
-MEPProvider          INFO Providing banks for VP ECal ODIN FT Muon UT
+MEPProvider          INFO Providing banks for VP UT FT Muon ODIN ECal
 MEPProvider          INFO Providing events in
 ApplicationMgr       INFO Application Manager Initialized successfully
 MEPProvider          INFO Opened mdf:root://eoslhcb.cern.ch//eos/lhcb/wg/rta/samples/mc/Upgrade_minbias_MD_FTv6_MEP/00146082_00000001_1.mep
-- 
GitLab


From 164e46e74efd615814195840f8ad099003fe2c58 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 24 May 2022 21:57:23 +0200
Subject: [PATCH 106/120] Remove obsolete test

---
 AllenOnline/tests/qmtest/mep_input.qmt | 34 --------------------------
 1 file changed, 34 deletions(-)
 delete mode 100644 AllenOnline/tests/qmtest/mep_input.qmt

diff --git a/AllenOnline/tests/qmtest/mep_input.qmt b/AllenOnline/tests/qmtest/mep_input.qmt
deleted file mode 100644
index 16bdf4ebc..000000000
--- a/AllenOnline/tests/qmtest/mep_input.qmt
+++ /dev/null
@@ -1,34 +0,0 @@
-<?xml version="1.0" ?><!DOCTYPE extension  PUBLIC '-//QM/2.3/Extension//EN'  'http://www.codesourcery.com/qm/dtds/2.3/-//qm/2.3/extension//en.dtd'>
-<!--
-    (c) Copyright 2020 CERN for the benefit of the LHCb Collaboration
--->
-<!--
-#######################################################
-# SUMMARY OF THIS TEST
-# ...................
-# Author: Roel Aaij
-# Purpose: Use ROOT python bindings to obtain the geometry directly
-#          from the stack and run the Allen event loop
-#######################################################
--->
-<extension class="GaudiTest.GaudiExeTest" kind="test">
-  <argument name="program"><text>python</text></argument>
-  <argument name="args"><set>
-    <text>${BINARYDUMPERSROOT}/options/allen.py</text>
-    <text>-n</text><text>10</text>
-    <text>--monitoring-filename</text><text>mep_input_hists.root</text>
-    <text>--mep</text><text>${ALLEN_PROJECT_ROOT}/input/minbias/mep/upgrade_mc_minbias_scifi_v5_pf10_retinacluster.mep</text>
-    <text>--mep-mask-source-id-top-5</text>
-  </set></argument>
-  <argument name="reference"><text>${ALLENROOT}/tests/refs/allen_10ev.ref</text></argument>
-  <arguement name="timeout"><integer>600</integer></arguement>
-  <argument name="validator"><text>
-
-from Allen.qmtest.exclusions import preprocessor
-from AllenOnline.qmtest.exclusions import skip_counters
-validateWithReference(preproc=preprocessor, counter_preproc=skip_counters)
-
-countErrorLines({"FATAL":0,"ERROR":0,"WARNING":2})
-
-  </text></argument>
-</extension>
-- 
GitLab


From 35138231579f2b513bb903bc65eeb5e7b6b0c127 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 24 May 2022 21:58:40 +0200
Subject: [PATCH 107/120] Add test to check for duplicate events written by
 Allen

---
 AllenOnline/tests/options/check_duplicates.py | 72 +++++++++++++++++++
 AllenOnline/tests/qmtest/check_duplicates.qmt | 29 ++++++++
 2 files changed, 101 insertions(+)
 create mode 100644 AllenOnline/tests/options/check_duplicates.py
 create mode 100644 AllenOnline/tests/qmtest/check_duplicates.qmt

diff --git a/AllenOnline/tests/options/check_duplicates.py b/AllenOnline/tests/options/check_duplicates.py
new file mode 100644
index 000000000..ccb734b60
--- /dev/null
+++ b/AllenOnline/tests/options/check_duplicates.py
@@ -0,0 +1,72 @@
+###############################################################################
+# (c) Copyright 2000-2018 CERN for the benefit of the LHCb Collaboration      #
+###############################################################################
+import sys
+from Configurables import LHCbApp, CondDB
+from Configurables import GaudiSequencer
+from Configurables import ApplicationMgr
+from Configurables import (AuditorSvc, SequencerTimerTool)
+from Configurables import IODataManager
+from Configurables import createODIN
+from Configurables import LHCb__UnpackRawEvent as UnpackRawEvent
+from GaudiConf import IOHelper
+
+app = LHCbApp(
+    DataType="Upgrade",
+    EvtMax=-1,
+    Simulation=True,
+    DDDBtag="dddb-20171122",
+    CondDBtag="sim-20180530-vc-md100")
+
+# Upgrade DBs
+CondDB().Upgrade = True
+
+# Decode VP, UT, FT and muons
+
+check_seq = GaudiSequencer("CheckODINSeq")
+
+unpack_raw = UnpackRawEvent(
+    RawEventLocation='DAQ/RawEvent',
+    RawBankLocations=['DAQ/RawBanks/ODIN'],
+    BankTypes=['ODIN'])
+
+check_seq.Members = [
+    unpack_raw, createODIN()
+]
+
+ApplicationMgr().TopAlg = [check_seq]
+
+IOHelper('MDF').inputFiles(['mep_passthrough.mdf'], clear=True)
+
+# Some extra stuff for timing table
+ApplicationMgr().ExtSvc += ['ToolSvc', 'AuditorSvc']
+ApplicationMgr().AuditAlgorithms = True
+AuditorSvc().Auditors += ['TimingAuditor']
+SequencerTimerTool().OutputLevel = 4
+
+# Some extra stuff to save histograms
+ApplicationMgr().HistogramPersistency = "NONE"
+
+# No error messages when reading MDF
+IODataManager().DisablePFNWarning = True
+
+from GaudiPython.Bindings import AppMgr, gbl
+gaudi = AppMgr()
+gaudi.initialize()
+TES = gaudi.evtSvc()
+
+events = set()
+
+n_evt = 0
+while app.EvtMax == -1 or n_evt < app.EvtMax:
+    gaudi.run(1)
+    if not TES['/Event']:
+        break
+
+    n_evt += 1
+
+    odin = TES['DAQ/ODIN']
+    events.add((odin.runNumber(), odin.eventNumber()))
+
+if len(events) != n_evt:
+    sys.exit("Found duplicate events")
diff --git a/AllenOnline/tests/qmtest/check_duplicates.qmt b/AllenOnline/tests/qmtest/check_duplicates.qmt
new file mode 100644
index 000000000..47fd22d89
--- /dev/null
+++ b/AllenOnline/tests/qmtest/check_duplicates.qmt
@@ -0,0 +1,29 @@
+<?xml version="1.0" ?><!DOCTYPE extension  PUBLIC '-//QM/2.3/Extension//EN'  'http://www.codesourcery.com/qm/dtds/2.3/-//qm/2.3/extension//en.dtd'>
+<!--
+    (c) Copyright 2020 CERN for the benefit of the LHCb Collaboration
+-->
+<!--
+#######################################################
+# SUMMARY OF THIS TEST
+# ...................
+# Author: Roel Aaij
+# Purpose: Use ROOT python bindings to obtain the geometry directly
+#          from the stack and run the Allen event loop on MEP input
+#######################################################
+-->
+<extension class="GaudiTest.GaudiExeTest" kind="test">
+  <argument name="program"><text>python</text></argument>
+  <argument name="args"><set>
+    <text>${ALLENONLINEROOT}/tests/options/check_duplicates.py</text>
+  </set></argument>
+  <argument name="prerequisites"><set>
+    <tuple><text>mep_passthrough</text><enumeral>PASS</enumeral></tuple>
+  </set></argument>
+  <argument name="use_temp_dir"><enumeral>true</enumeral></argument>
+  <argument name="timeout"><integer>600</integer></argument>
+  <argument name="validator"><text>
+
+countErrorLines({"FATAL":0,"ERROR":0,"WARNING":0})
+
+  </text></argument>
+</extension>
-- 
GitLab


From 8d963b5f075024501d35b6305d6d29fee10dc38e Mon Sep 17 00:00:00 2001
From: Gitlab CI <noreply@cern.ch>
Date: Tue, 24 May 2022 19:59:06 +0000
Subject: [PATCH 108/120] Fixed formatting

patch generated by https://gitlab.cern.ch/lhcb/MooreOnline/-/jobs/22052592
---
 AllenOnline/include/AllenOnline/MPIConfig.h    |  2 +-
 AllenOnline/include/AllenOnline/TransposeMEP.h |  1 -
 AllenOnline/src/AllenApplication.cpp           |  3 +--
 AllenOnline/src/AllenApplication.h             |  2 +-
 AllenOnline/src/MEPProvider.cpp                | 18 ++++++++----------
 AllenOnline/src/MEPProvider.h                  |  2 +-
 AllenOnline/src/TransposeMEP.cpp               |  2 +-
 AllenOnline/tests/options/check_duplicates.py  |  4 +---
 8 files changed, 14 insertions(+), 20 deletions(-)

diff --git a/AllenOnline/include/AllenOnline/MPIConfig.h b/AllenOnline/include/AllenOnline/MPIConfig.h
index 163f93b7b..72f7d5df1 100644
--- a/AllenOnline/include/AllenOnline/MPIConfig.h
+++ b/AllenOnline/include/AllenOnline/MPIConfig.h
@@ -6,7 +6,7 @@
 #include <limits.h>
 
 #ifdef HAVE_MPI
-#include <mpi.h>
+#  include <mpi.h>
 #endif
 
 // Determine size of size_t for MPI type
diff --git a/AllenOnline/include/AllenOnline/TransposeMEP.h b/AllenOnline/include/AllenOnline/TransposeMEP.h
index 2cca5a8f1..f120d9a3a 100644
--- a/AllenOnline/include/AllenOnline/TransposeMEP.h
+++ b/AllenOnline/include/AllenOnline/TransposeMEP.h
@@ -67,7 +67,6 @@ namespace MEP {
   LHCb::ODIN decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes,
                           const uint8_t version );
 
-
   /**
    * @brief      Fill the array the contains the number of banks per type
    *
diff --git a/AllenOnline/src/AllenApplication.cpp b/AllenOnline/src/AllenApplication.cpp
index 69ce1d956..394618eb9 100644
--- a/AllenOnline/src/AllenApplication.cpp
+++ b/AllenOnline/src/AllenApplication.cpp
@@ -289,8 +289,7 @@ void AllenApplication::allenLoop() {
   allen( m_options, this, m_provider, m_output, m_zmqSvc.get(), m_controlConnection );
 }
 
-void AllenApplication::update(gsl::span<unsigned const> odin_data)
-{
+void AllenApplication::update( gsl::span<unsigned const> odin_data ) {
   LHCb::ODIN odin{odin_data};
   auto const run = odin.runNumber();
 
diff --git a/AllenOnline/src/AllenApplication.h b/AllenOnline/src/AllenApplication.h
index b2123e461..30ac64466 100644
--- a/AllenOnline/src/AllenApplication.h
+++ b/AllenOnline/src/AllenApplication.h
@@ -66,7 +66,7 @@ public:
   bool initMPI();
 
   // Updater
-  void update(gsl::span<unsigned const> odin_data) override;
+  void update( gsl::span<unsigned const> odin_data ) override;
 
   void registerConsumer( std::string const& id, std::unique_ptr<Allen::NonEventData::Consumer> c ) override;
 
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
index faccaf072..14b0416da 100644
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -6,10 +6,10 @@
 #include <Allen/BankTypes.h>
 #include <Allen/InputProvider.h>
 #include <Allen/Logger.h>
+#include <Allen/MEPTools.h>
 #include <Allen/Provider.h>
 #include <Allen/SliceUtils.h>
 #include <Allen/Timer.h>
-#include <Allen/MEPTools.h>
 #include <Allen/sourceid.h>
 #include <Allen/write_mdf.hpp>
 #include <Backend/BackendCommon.h>
@@ -113,15 +113,13 @@ std::tuple<bool, bool, bool, size_t, size_t, std::any> MEPProvider::get_slice( s
       std::tie( slice_index, n_filled ) = m_transposed.front();
       m_transposed.pop_front();
       if ( n_filled > 0 ) {
-        auto bno = banks(BankTypes::ODIN, slice_index);
-        auto const* block = bno.fragments[0].data();
+        auto        bno     = banks( BankTypes::ODIN, slice_index );
+        auto const* block   = bno.fragments[0].data();
         auto const* offsets = bno.offsets.data();
-        auto const n_banks = offsets[0];
-        auto odin = MEP::decode_odin( block,
-                                      offsets[MEP::offset_index(n_banks, 0, 0)],
-                                      MEP::bank_size(block, bno.sizes.data(), 0, 0),
-                                      bno.version );
-        odin_span = gsl::span<unsigned const>{odin.data};
+        auto const  n_banks = offsets[0];
+        auto        odin    = MEP::decode_odin( block, offsets[MEP::offset_index( n_banks, 0, 0 )],
+                                      MEP::bank_size( block, bno.sizes.data(), 0, 0 ), bno.version );
+        odin_span           = gsl::span<unsigned const>{odin.data};
       }
     }
   }
@@ -272,7 +270,7 @@ StatusCode MEPProvider::initialize() {
   } else {
     info() << "Providing banks for";
     std::vector<BankTypes> types{m_bank_types.begin(), m_bank_types.end()};
-    std::sort(types.begin(), types.end());
+    std::sort( types.begin(), types.end() );
     for ( auto bt : types ) info() << " " << ::bank_name( bt );
     info() << endmsg;
   }
diff --git a/AllenOnline/src/MEPProvider.h b/AllenOnline/src/MEPProvider.h
index 7f1de6c5e..1af45a683 100644
--- a/AllenOnline/src/MEPProvider.h
+++ b/AllenOnline/src/MEPProvider.h
@@ -4,6 +4,7 @@
 #pragma once
 
 #include <algorithm>
+#include <any>
 #include <array>
 #include <atomic>
 #include <cassert>
@@ -14,7 +15,6 @@
 #include <numeric>
 #include <thread>
 #include <vector>
-#include <any>
 
 #include <fcntl.h>
 #include <sys/stat.h>
diff --git a/AllenOnline/src/TransposeMEP.cpp b/AllenOnline/src/TransposeMEP.cpp
index bd33f582e..63e7cdd0c 100644
--- a/AllenOnline/src/TransposeMEP.cpp
+++ b/AllenOnline/src/TransposeMEP.cpp
@@ -44,7 +44,7 @@ MEP::fill_counts( EB::MEP const* mep ) {
 LHCb::ODIN MEP::decode_odin( char const* odin_data, unsigned const offset, unsigned const size_bytes,
                              const uint8_t version ) {
   LHCb::span<std::uint32_t const> odin_span{reinterpret_cast<std::uint32_t const*>( odin_data + offset ),
-    size_bytes / sizeof( uint32_t )};
+                                            size_bytes / sizeof( uint32_t )};
   if ( version == 6 ) {
     return LHCb::ODIN::from_version<6>( odin_span );
   } else {
diff --git a/AllenOnline/tests/options/check_duplicates.py b/AllenOnline/tests/options/check_duplicates.py
index ccb734b60..753bfd1fa 100644
--- a/AllenOnline/tests/options/check_duplicates.py
+++ b/AllenOnline/tests/options/check_duplicates.py
@@ -30,9 +30,7 @@ unpack_raw = UnpackRawEvent(
     RawBankLocations=['DAQ/RawBanks/ODIN'],
     BankTypes=['ODIN'])
 
-check_seq.Members = [
-    unpack_raw, createODIN()
-]
+check_seq.Members = [unpack_raw, createODIN()]
 
 ApplicationMgr().TopAlg = [check_seq]
 
-- 
GitLab


From 239cfbeabacf620b8ae197fa663237ec033e5996 Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Wed, 25 May 2022 18:05:24 +0200
Subject: [PATCH 109/120] update AllenConfig.py following DD4Hep merge request

---
 AllenOnline/options/AllenConfig.py | 44 +++++++++++++++++++++---------
 1 file changed, 31 insertions(+), 13 deletions(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 6a2b59926..3713b2c92 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -22,6 +22,8 @@ from Configurables import MonitorSvc
 from Configurables import Online__Configuration as OnlineConfiguration
 from Configurables import MEPProvider, MessageSvc
 from Allen.config import setup_allen_non_event_data_service
+from PyConf.application import ApplicationOptions, ComponentConfig, setup_component, configure
+from Configurables import Gaudi__RootCnvSvc as RootCnvSvc
 
 try:
     import OnlineEnvBase as OnlineEnv
@@ -42,22 +44,19 @@ except ImportError:
 
 integration_test = False
 
-app = LHCbApp(
-    DataType="Upgrade",
-    Simulation=True,
-    DDDBtag=dddb_tag,
-    CondDBtag=conddb_tag)
-# DDDBtag="dddb-20210218",
-# CondDBtag="sim-20201218-vc-md100")
+options = ApplicationOptions(_enabled=False)
+options.simulation = True
+options.data_type = 'Upgrade'
+options.input_type = 'MDF'
+options.dddb_tag = dddb_tag
+options.conddb_tag = conddb_tag
 
-setup_allen_non_event_data_service()
+options.finalize()
+config = ComponentConfig()
 
 appMgr = ApplicationMgr()
 appMgr.AppName = ""
 
-# Upgrade DBs
-CondDB().Upgrade = True
-
 # Get the number of lines from the configuration. This should probably
 # be done internally to Allen and retrieved somehow
 n_lines = 0
@@ -166,7 +165,26 @@ appMgr.ExtSvc = [monSvc, "OnlMonitorSink"] + appMgr.ExtSvc
 appMgr.EvtSel = "NONE"
 
 # Key services, order matters!
+rootSvc = RootCnvSvc("RootCnvSvc", EnableIncident=1)
 appMgr.ExtSvc = [
-    'ToolSvc', 'AuditorSvc', allen_conf, 'Online::Configuration/Application',
-    'ZeroMQSvc', 'MEPProvider'
+    'ToolSvc', 'AuditorSvc', 'ZeroMQSvc', 
+    "Gaudi::IODataManager/IODataManager", rootSvc,
+    allen_conf, 'Online::Configuration/Application',
+    'MEPProvider'
 ] + appMgr.ExtSvc
+
+# Copeid from PyConf.application.configure_input
+config.add(
+    setup_component(
+        'DDDBConf', Simulation=options.simulation, DataType=options.data_type))
+config.add(
+    setup_component(
+        'CondDB',
+        Upgrade=True,
+        Tags={
+            'DDDB': options.dddb_tag,
+            'SIMCOND': options.conddb_tag,
+        }))
+
+cf_node = setup_allen_non_event_data_service(allen_event_loop=True)
+config.update(configure(options, cf_node))
-- 
GitLab


From 363ba2da1c6376ca35f82db6827e95d74ed14ce0 Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Wed, 25 May 2022 20:03:16 +0200
Subject: [PATCH 110/120] fix monitor service configuration

---
 AllenOnline/options/AllenConfig.py | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 3713b2c92..8e2c4bc83 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -13,7 +13,7 @@ import os
 import re
 import json
 from Gaudi.Configuration import importOptions, allConfigurables
-from Configurables import LHCbApp, CondDB, ApplicationMgr
+from Configurables import LHCbApp, CondDB, ApplicationMgr, OnlMonitorSink
 from Configurables import DumpUTGeometry, DumpFTGeometry, DumpMuonTable
 from Configurables import DumpMuonGeometry, DumpVPGeometry, AllenUpdater
 from Configurables import DumpMagneticField, DumpBeamline, DumpUTLookupTables
@@ -160,7 +160,10 @@ messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
 messageSvc.OutputLevel = 3
 
 # Add the services that will produce the non-event-data
-appMgr.ExtSvc = [monSvc, "OnlMonitorSink"] + appMgr.ExtSvc
+monSink = OnlMonitorSink(         
+    CountersToPublish=[("Bursts", "IN")],         
+    HistogramsToPublish=[(".*", ".*")])
+appMgr.ExtSvc = [monSvc, monSink] + appMgr.ExtSvc
 
 appMgr.EvtSel = "NONE"
 
-- 
GitLab


From aa2bb06d864bf60b48b072f9f58987970ad7dc30 Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Thu, 26 May 2022 10:11:39 +0200
Subject: [PATCH 111/120] Added all counters

---
 AllenOnline/options/AllenConfig.py | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index 8e2c4bc83..f48e6ecae 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -161,7 +161,9 @@ messageSvc.OutputLevel = 3
 
 # Add the services that will produce the non-event-data
 monSink = OnlMonitorSink(         
-    CountersToPublish=[("Bursts", "IN")],         
+    CountersToPublish=[("Bursts", "IN"), ("Bursts", "OUT"),
+                       ("Events", "IN"), ("Events", "OUT"),
+                       ("Events", "MB_IN"), ("Events", "MB_OUT")],
     HistogramsToPublish=[(".*", ".*")])
 appMgr.ExtSvc = [monSvc, monSink] + appMgr.ExtSvc
 
-- 
GitLab


From 0919b2d3ba6f5aaa5b468d7b7ba21ea2e2e5a876 Mon Sep 17 00:00:00 2001
From: Dorothea vom Bruch <dorothea.vom.bruch@cern.ch>
Date: Fri, 27 May 2022 11:37:26 +0200
Subject: [PATCH 112/120] bursts svc fix

---
 AllenOnline/src/MBMOutput.cpp   | 2 +-
 AllenOnline/src/MEPProvider.cpp | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
 mode change 100644 => 100755 AllenOnline/src/MBMOutput.cpp
 mode change 100644 => 100755 AllenOnline/src/MEPProvider.cpp

diff --git a/AllenOnline/src/MBMOutput.cpp b/AllenOnline/src/MBMOutput.cpp
old mode 100644
new mode 100755
index 9c668c112..2d3b887db
--- a/AllenOnline/src/MBMOutput.cpp
+++ b/AllenOnline/src/MBMOutput.cpp
@@ -161,7 +161,7 @@ StatusCode Allen::MBMOutput::initialize() {
   }
 
   auto burstsSvc = dynamic_cast<Service*>( service<IService>( "AllenIOMon/Bursts", true ).get() );
-  if ( !eventsSvc ) {
+  if ( !burstsSvc ) {
     error() << "Failed to obtain Bursts service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
diff --git a/AllenOnline/src/MEPProvider.cpp b/AllenOnline/src/MEPProvider.cpp
old mode 100644
new mode 100755
index 14b0416da..070d8d429
--- a/AllenOnline/src/MEPProvider.cpp
+++ b/AllenOnline/src/MEPProvider.cpp
@@ -282,7 +282,7 @@ StatusCode MEPProvider::initialize() {
   }
 
   auto burstsSvc = dynamic_cast<Service*>( service<IService>( "AllenIOMon/Bursts", true ).get() );
-  if ( !eventsSvc ) {
+  if ( !burstsSvc ) {
     error() << "Failed to obtain Bursts service for monitoring" << endmsg;
     return StatusCode::FAILURE;
   }
-- 
GitLab


From d6c71c23db714d38b0af06a00de7629143d3d856 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Thu, 26 May 2022 11:15:01 +0200
Subject: [PATCH 113/120] Update reference following changes to run changes in
 Allen

---
 AllenOnline/tests/refs/mep_passthrough.ref | 1 -
 1 file changed, 1 deletion(-)

diff --git a/AllenOnline/tests/refs/mep_passthrough.ref b/AllenOnline/tests/refs/mep_passthrough.ref
index ccc6f3123..ccf856413 100644
--- a/AllenOnline/tests/refs/mep_passthrough.ref
+++ b/AllenOnline/tests/refs/mep_passthrough.ref
@@ -20,7 +20,6 @@ ApplicationMgr                         INFO Application Manager Initialized succ
 DeFTDetector                           INFO Current FT geometry version =   64
 ApplicationMgr                         INFO Application Manager Started successfully
 Setting number of slices to 4
-DumpFTGeometry                         INFO Conditions DB is compatible with FT bank version 4, 5, 6.
 Input complete
 Processing complete
 Wrote 2000/2000 events to mep_passthrough.mdf
-- 
GitLab


From 95420262a84182da1b0b6c2e06272ec4f4810aea Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Sun, 29 May 2022 16:54:29 +0200
Subject: [PATCH 114/120] Update check_duplicates to also check HltDecReports
 and total number of events

---
 AllenOnline/tests/options/check_duplicates.py | 75 +++++++++++++++++--
 AllenOnline/tests/qmtest/check_duplicates.qmt |  3 +
 AllenOnline/tests/qmtest/mep_passthrough.qmt  |  6 +-
 AllenOnline/tests/refs/mep_passthrough.ref    | 43 ++++++++++-
 4 files changed, 117 insertions(+), 10 deletions(-)

diff --git a/AllenOnline/tests/options/check_duplicates.py b/AllenOnline/tests/options/check_duplicates.py
index 753bfd1fa..e032590b1 100644
--- a/AllenOnline/tests/options/check_duplicates.py
+++ b/AllenOnline/tests/options/check_duplicates.py
@@ -1,15 +1,58 @@
 ###############################################################################
 # (c) Copyright 2000-2018 CERN for the benefit of the LHCb Collaboration      #
 ###############################################################################
+import os
 import sys
+import re
+import argparse
+from collections import defaultdict
 from Configurables import LHCbApp, CondDB
 from Configurables import GaudiSequencer
 from Configurables import ApplicationMgr
 from Configurables import (AuditorSvc, SequencerTimerTool)
 from Configurables import IODataManager
-from Configurables import createODIN
+from Configurables import createODIN, HltDecReportsDecoder
 from Configurables import LHCb__UnpackRawEvent as UnpackRawEvent
 from GaudiConf import IOHelper
+from RecoConf.hlt1_allen import sequence
+from Moore.config import get_allen_hlt1_decision_ids, setup_ann_service
+from GaudiPython.Bindings import AppMgr, gbl
+
+parser = argparse.ArgumentParser()
+parser.add_argument("mdf", nargs=1)
+parser.add_argument("ref", nargs=1)
+parser.add_argument("sequence", nargs=1)
+args = parser.parse_args()
+
+rate_lines = []
+add_rate = False
+with open(args.ref[0]) as ref_file:
+    for line in ref_file:
+        line = line.strip()
+        if line.startswith("rate_validator"):
+            add_rate = True
+            continue
+        if add_rate:
+            rate_lines.append(line)
+        if line.startswith('Inclusive:'):
+            add_rate = False
+            break
+
+rates = {}
+rate_expr = re.compile(r"([^:]+):[ ]+([0-9]+)/[ ]+([0-9]+)")
+for rate_line in rate_lines:
+    m = rate_expr.search(rate_line)
+    rates[m.group(1)] = int(m.group(2))
+
+
+sequence_file = os.path.expandvars(args.sequence[0])
+sequence_name = os.path.basename(sequence_file)
+sequence_name = os.path.splitext(sequence_name)[0]
+
+with sequence.bind(sequence=sequence_name, json=sequence_file):
+    decs = get_allen_hlt1_decision_ids()
+    ann_svc = setup_ann_service(decs, {}, {})
+
 
 app = LHCbApp(
     DataType="Upgrade",
@@ -30,11 +73,17 @@ unpack_raw = UnpackRawEvent(
     RawBankLocations=['DAQ/RawBanks/ODIN'],
     BankTypes=['ODIN'])
 
-check_seq.Members = [unpack_raw, createODIN()]
+
+dec_reports = HltDecReportsDecoder(ANNSvc=ann_svc.getFullName(),
+                                   RawEventLocations='DAQ/RawEvent',
+                                   SourceID='Hlt1',
+                                   OutputHltDecReportsLocation='Hlt1/DecReports')
+
+check_seq.Members = [unpack_raw, createODIN(), dec_reports]
 
 ApplicationMgr().TopAlg = [check_seq]
 
-IOHelper('MDF').inputFiles(['mep_passthrough.mdf'], clear=True)
+IOHelper('MDF').inputFiles(args.mdf, clear=True)
 
 # Some extra stuff for timing table
 ApplicationMgr().ExtSvc += ['ToolSvc', 'AuditorSvc']
@@ -48,13 +97,14 @@ ApplicationMgr().HistogramPersistency = "NONE"
 # No error messages when reading MDF
 IODataManager().DisablePFNWarning = True
 
-from GaudiPython.Bindings import AppMgr, gbl
 gaudi = AppMgr()
 gaudi.initialize()
 TES = gaudi.evtSvc()
 
 events = set()
 
+decs = defaultdict(int)
+
 n_evt = 0
 while app.EvtMax == -1 or n_evt < app.EvtMax:
     gaudi.run(1)
@@ -66,5 +116,20 @@ while app.EvtMax == -1 or n_evt < app.EvtMax:
     odin = TES['DAQ/ODIN']
     events.add((odin.runNumber(), odin.eventNumber()))
 
+    reps = TES['Hlt1/DecReports']
+    for n in reps.decisionNames():
+        decs[str(n)[:-8]] += reps.decReport(n).decision()
+
 if len(events) != n_evt:
-    sys.exit("Found duplicate events")
+    sys.exit("Found %d duplicate events" % (n_evt - len(events)))
+elif n_evt != rates['Inclusive']:
+    sys.exit("Wrong number of events %d, but should be %d" % (n_evt, rates['Inclusive']))
+
+wrong_decs = False
+for n, n_dec in decs.items():
+    if rates[n] != n_dec:
+        wrong_decs = True
+        print("Wrong number of decisions for %d: %d should be %d" % (n, n_dec, rates[n]))
+
+if wrong_decs:
+    sys.exit("Wrong number of decisions")
diff --git a/AllenOnline/tests/qmtest/check_duplicates.qmt b/AllenOnline/tests/qmtest/check_duplicates.qmt
index 47fd22d89..aa6c5c0f2 100644
--- a/AllenOnline/tests/qmtest/check_duplicates.qmt
+++ b/AllenOnline/tests/qmtest/check_duplicates.qmt
@@ -15,6 +15,9 @@
   <argument name="program"><text>python</text></argument>
   <argument name="args"><set>
     <text>${ALLENONLINEROOT}/tests/options/check_duplicates.py</text>
+    <text>mep_passthrough.mdf</text>
+    <text>${ALLENONLINEROOT}/tests/refs/mep_passthrough.ref</text>
+    <text>${ALLEN_INSTALL_DIR}/constants/hlt1_pp_default.json</text>
   </set></argument>
   <argument name="prerequisites"><set>
     <tuple><text>mep_passthrough</text><enumeral>PASS</enumeral></tuple>
diff --git a/AllenOnline/tests/qmtest/mep_passthrough.qmt b/AllenOnline/tests/qmtest/mep_passthrough.qmt
index 707bbc0eb..c003a4b80 100644
--- a/AllenOnline/tests/qmtest/mep_passthrough.qmt
+++ b/AllenOnline/tests/qmtest/mep_passthrough.qmt
@@ -28,11 +28,11 @@
   </set></argument>
   <argument name="use_temp_dir"><enumeral>true</enumeral></argument>
   <argument name="reference"><text>../refs/mep_passthrough.ref</text></argument>
-  <argument name="timeout"><integer>600</integer></argument>
+  <argument name="timeout"><integer>3000</integer></argument>
   <argument name="validator"><text>
 
-from Allen.qmtest.exclusions import preprocessor
-validateWithReference(preproc = preprocessor)
+from Allen.qmtest.exclusions import preprocessor_with_rates
+validateWithReference(preproc = preprocessor_with_rates)
 
 countErrorLines({"FATAL":0,"ERROR":0,"WARNING":0})
 
diff --git a/AllenOnline/tests/refs/mep_passthrough.ref b/AllenOnline/tests/refs/mep_passthrough.ref
index ccf856413..d4cb4dd16 100644
--- a/AllenOnline/tests/refs/mep_passthrough.ref
+++ b/AllenOnline/tests/refs/mep_passthrough.ref
@@ -17,12 +17,15 @@ HLTControlFlowMgr                      INFO Concurrency level information:
 HLTControlFlowMgr                      INFO  o Number of events slots: 1
 HLTControlFlowMgr                      INFO  o TBB thread pool size:  'ThreadPoolSize':1
 ApplicationMgr                         INFO Application Manager Initialized successfully
+Setting number of slices to 4
+Connecting allen_control to inproc:///py_allen_control
 DeFTDetector                           INFO Current FT geometry version =   64
 ApplicationMgr                         INFO Application Manager Started successfully
-Setting number of slices to 4
+DumpFTGeometry                         INFO Conditions DB is compatible with FT bank version 4, 5, 6.
+Starting timer for throughput measurement
 Input complete
 Processing complete
-Wrote 2000/2000 events to mep_passthrough.mdf
+Processing complete
 ApplicationMgr                         INFO Application Manager Stopped successfully
 HLTControlFlowMgr                      INFO
 HLTControlFlowMgr                      INFO
@@ -42,6 +45,42 @@ LAZY_AND: allen_non_event_data                #=1       Sum=1           Eff=|( 1
 HLTControlFlowMgr                      INFO Histograms converted successfully according to request.
 ToolSvc                                INFO Removing all tools created by ToolSvc
 ApplicationMgr                         INFO Application Manager Finalized successfully
+Processing complete
+rate_validator validation:
+Hlt1KsToPiPi_gec:                   28/  2000, (  420.00 +/-    78.81) kHz
+Hlt1TrackMVA_gec:                   22/  2000, (  330.00 +/-    69.97) kHz
+Hlt1TwoTrackMVA_gec:                41/  2000, (  615.00 +/-    95.06) kHz
+Hlt1TwoTrackKs_gec:                  3/  2000, (   45.00 +/-    25.96) kHz
+Hlt1SingleHighPtMuon_gec:            0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1SingleHighPtMuonNoMuID_gec:      1/  2000, (   15.00 +/-    15.00) kHz
+Hlt1LowPtMuon_gec:                 176/  2000, ( 2640.00 +/-   190.04) kHz
+Hlt1D2KK_gec:                        5/  2000, (   75.00 +/-    33.50) kHz
+Hlt1D2KPi_gec:                       9/  2000, (  135.00 +/-    44.90) kHz
+Hlt1D2PiPi_gec:                      4/  2000, (   60.00 +/-    29.97) kHz
+Hlt1DiMuonHighMass_gec:             19/  2000, (  285.00 +/-    65.07) kHz
+Hlt1DiMuonLowMass_gec:              63/  2000, (  945.00 +/-   117.17) kHz
+Hlt1DiMuonSoft_gec:                  0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1LowPtDiMuon_gec:                78/  2000, ( 1170.00 +/-   129.87) kHz
+Hlt1TrackMuonMVA_gec:                0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1TrackElectronMVA_gec:            8/  2000, (  120.00 +/-    42.34) kHz
+Hlt1SingleHighPtElectron_gec:        2/  2000, (   30.00 +/-    21.20) kHz
+Hlt1DisplacedDielectron_gec:         6/  2000, (   90.00 +/-    36.69) kHz
+Hlt1DisplacedLeptons_gec:           15/  2000, (  225.00 +/-    57.88) kHz
+Hlt1SingleHighEt_gec:                0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1Passthrough:                  2000/  2000, (30000.00 +/-     0.00) kHz
+Hlt1Passthrough_gec:              1789/  2000, (26835.00 +/-   206.07) kHz
+Hlt1NoBeam:                          0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BeamOne:                         0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BeamTwo:                         0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BothBeams:                       0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1ODINLumi:                        0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1ODINNoBias:                      0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1VeloMicroBias_gec:               3/  2000, (   45.00 +/-    25.96) kHz
+Hlt1RICH1Alignment_gec:              1/  2000, (   15.00 +/-    15.00) kHz
+Hlt1RICH2Alignment_gec:              0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BeamGas_gec:                     0/  2000, (    0.00 +/-     0.00) kHz
+Inclusive:                        2000/  2000, (30000.00 +/-     0.00) kHz
+Wrote 2000/2000 events to mep_passthrough.mdf
 ApplicationMgr                         INFO Application Manager Terminated successfully
 Bursts                                 INFO Number of counters : 1
  |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
-- 
GitLab


From a376eb18c8b9c7d373f0fd0a92ee062e6c159331 Mon Sep 17 00:00:00 2001
From: Gitlab CI <noreply@cern.ch>
Date: Sun, 29 May 2022 20:26:05 +0000
Subject: [PATCH 115/120] Fixed formatting

patch generated by https://gitlab.cern.ch/lhcb/MooreOnline/-/jobs/22137970
---
 AllenOnline/options/AllenConfig.py            | 14 ++++++--------
 AllenOnline/tests/options/check_duplicates.py | 18 +++++++++---------
 2 files changed, 15 insertions(+), 17 deletions(-)

diff --git a/AllenOnline/options/AllenConfig.py b/AllenOnline/options/AllenConfig.py
index f48e6ecae..0b188a89b 100755
--- a/AllenOnline/options/AllenConfig.py
+++ b/AllenOnline/options/AllenConfig.py
@@ -160,10 +160,10 @@ messageSvc.Format = '% F%8W%L%T %25W%L%S %0W%M'
 messageSvc.OutputLevel = 3
 
 # Add the services that will produce the non-event-data
-monSink = OnlMonitorSink(         
-    CountersToPublish=[("Bursts", "IN"), ("Bursts", "OUT"),
-                       ("Events", "IN"), ("Events", "OUT"),
-                       ("Events", "MB_IN"), ("Events", "MB_OUT")],
+monSink = OnlMonitorSink(
+    CountersToPublish=[("Bursts", "IN"), ("Bursts", "OUT"), ("Events", "IN"),
+                       ("Events", "OUT"), ("Events", "MB_IN"),
+                       ("Events", "MB_OUT")],
     HistogramsToPublish=[(".*", ".*")])
 appMgr.ExtSvc = [monSvc, monSink] + appMgr.ExtSvc
 
@@ -172,10 +172,8 @@ appMgr.EvtSel = "NONE"
 # Key services, order matters!
 rootSvc = RootCnvSvc("RootCnvSvc", EnableIncident=1)
 appMgr.ExtSvc = [
-    'ToolSvc', 'AuditorSvc', 'ZeroMQSvc', 
-    "Gaudi::IODataManager/IODataManager", rootSvc,
-    allen_conf, 'Online::Configuration/Application',
-    'MEPProvider'
+    'ToolSvc', 'AuditorSvc', 'ZeroMQSvc', "Gaudi::IODataManager/IODataManager",
+    rootSvc, allen_conf, 'Online::Configuration/Application', 'MEPProvider'
 ] + appMgr.ExtSvc
 
 # Copeid from PyConf.application.configure_input
diff --git a/AllenOnline/tests/options/check_duplicates.py b/AllenOnline/tests/options/check_duplicates.py
index e032590b1..6e6a8b684 100644
--- a/AllenOnline/tests/options/check_duplicates.py
+++ b/AllenOnline/tests/options/check_duplicates.py
@@ -44,7 +44,6 @@ for rate_line in rate_lines:
     m = rate_expr.search(rate_line)
     rates[m.group(1)] = int(m.group(2))
 
-
 sequence_file = os.path.expandvars(args.sequence[0])
 sequence_name = os.path.basename(sequence_file)
 sequence_name = os.path.splitext(sequence_name)[0]
@@ -53,7 +52,6 @@ with sequence.bind(sequence=sequence_name, json=sequence_file):
     decs = get_allen_hlt1_decision_ids()
     ann_svc = setup_ann_service(decs, {}, {})
 
-
 app = LHCbApp(
     DataType="Upgrade",
     EvtMax=-1,
@@ -73,11 +71,11 @@ unpack_raw = UnpackRawEvent(
     RawBankLocations=['DAQ/RawBanks/ODIN'],
     BankTypes=['ODIN'])
 
-
-dec_reports = HltDecReportsDecoder(ANNSvc=ann_svc.getFullName(),
-                                   RawEventLocations='DAQ/RawEvent',
-                                   SourceID='Hlt1',
-                                   OutputHltDecReportsLocation='Hlt1/DecReports')
+dec_reports = HltDecReportsDecoder(
+    ANNSvc=ann_svc.getFullName(),
+    RawEventLocations='DAQ/RawEvent',
+    SourceID='Hlt1',
+    OutputHltDecReportsLocation='Hlt1/DecReports')
 
 check_seq.Members = [unpack_raw, createODIN(), dec_reports]
 
@@ -123,13 +121,15 @@ while app.EvtMax == -1 or n_evt < app.EvtMax:
 if len(events) != n_evt:
     sys.exit("Found %d duplicate events" % (n_evt - len(events)))
 elif n_evt != rates['Inclusive']:
-    sys.exit("Wrong number of events %d, but should be %d" % (n_evt, rates['Inclusive']))
+    sys.exit("Wrong number of events %d, but should be %d" %
+             (n_evt, rates['Inclusive']))
 
 wrong_decs = False
 for n, n_dec in decs.items():
     if rates[n] != n_dec:
         wrong_decs = True
-        print("Wrong number of decisions for %d: %d should be %d" % (n, n_dec, rates[n]))
+        print("Wrong number of decisions for %d: %d should be %d" % (n, n_dec,
+                                                                     rates[n]))
 
 if wrong_decs:
     sys.exit("Wrong number of decisions")
-- 
GitLab


From 08583dbdd19f04903ba33ec05a0f2c9d74f8fa70 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 30 May 2022 17:11:04 +0200
Subject: [PATCH 116/120] Update reference

---
 AllenOnline/tests/refs/mep_passthrough.ref | 4 ----
 1 file changed, 4 deletions(-)

diff --git a/AllenOnline/tests/refs/mep_passthrough.ref b/AllenOnline/tests/refs/mep_passthrough.ref
index d4cb4dd16..233eb4398 100644
--- a/AllenOnline/tests/refs/mep_passthrough.ref
+++ b/AllenOnline/tests/refs/mep_passthrough.ref
@@ -18,14 +18,11 @@ HLTControlFlowMgr                      INFO  o Number of events slots: 1
 HLTControlFlowMgr                      INFO  o TBB thread pool size:  'ThreadPoolSize':1
 ApplicationMgr                         INFO Application Manager Initialized successfully
 Setting number of slices to 4
-Connecting allen_control to inproc:///py_allen_control
 DeFTDetector                           INFO Current FT geometry version =   64
 ApplicationMgr                         INFO Application Manager Started successfully
 DumpFTGeometry                         INFO Conditions DB is compatible with FT bank version 4, 5, 6.
 Starting timer for throughput measurement
 Input complete
-Processing complete
-Processing complete
 ApplicationMgr                         INFO Application Manager Stopped successfully
 HLTControlFlowMgr                      INFO
 HLTControlFlowMgr                      INFO
@@ -45,7 +42,6 @@ LAZY_AND: allen_non_event_data                #=1       Sum=1           Eff=|( 1
 HLTControlFlowMgr                      INFO Histograms converted successfully according to request.
 ToolSvc                                INFO Removing all tools created by ToolSvc
 ApplicationMgr                         INFO Application Manager Finalized successfully
-Processing complete
 rate_validator validation:
 Hlt1KsToPiPi_gec:                   28/  2000, (  420.00 +/-    78.81) kHz
 Hlt1TrackMVA_gec:                   22/  2000, (  330.00 +/-    69.97) kHz
-- 
GitLab


From 224687232ebcb4857dcc1baa51c7b0484713d420 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Mon, 30 May 2022 17:11:18 +0200
Subject: [PATCH 117/120] Update configuration of UT decoding

---
 AllenOnline/tests/options/cpu_decoding.py | 26 ++++++++++++-----------
 1 file changed, 14 insertions(+), 12 deletions(-)

diff --git a/AllenOnline/tests/options/cpu_decoding.py b/AllenOnline/tests/options/cpu_decoding.py
index b9eb496a9..e0e14b532 100644
--- a/AllenOnline/tests/options/cpu_decoding.py
+++ b/AllenOnline/tests/options/cpu_decoding.py
@@ -43,20 +43,22 @@ def decoding():
 
     ecal_raw = default_raw_event(["EcalPacked"])
 
-    decoders = [
-        loc.producer for loc in [
-            make_VeloClusterTrackingSIMD_hits(),
-            make_PrStorePrUTHits_hits(),
-            make_PrStoreSciFiHits_hits(),
-            make_rich_pixels(default_rich_reco_options())['RichDecodedData'],
-            make_ecal_digits(ecal_raw),
-            make_muon_hits(),
-            make_decreports(),
-            make_selreports()
+    with make_PrStorePrUTHits_hits.bind(isCluster=False):
+        decoders = [
+            loc.producer for loc in [
+                make_VeloClusterTrackingSIMD_hits(),
+                make_PrStorePrUTHits_hits(),
+                make_PrStoreSciFiHits_hits(),
+                make_rich_pixels(default_rich_reco_options())
+                ['RichDecodedData'],
+                make_ecal_digits(ecal_raw),
+                make_muon_hits(),
+                make_decreports(),
+                make_selreports()
+            ]
         ]
-    ]
 
-    return Reconstruction('decoding', decoders, [])
+        return Reconstruction('decoding', decoders, [])
 
 
 run_reconstruction(options, decoding)
-- 
GitLab


From 190e6dd63747d239281d8997d5af5a8e7d037398 Mon Sep 17 00:00:00 2001
From: Roel Aaij <roel.aaij@cern.ch>
Date: Tue, 31 May 2022 15:00:13 +0200
Subject: [PATCH 118/120] Use x86_64_v3 reference for v3 platforms

---
 AllenOnline/tests/options/check_duplicates.py |  6 +-
 .../refs/mep_passthrough.ref.x86_64_v3-opt    | 87 +++++++++++++++++++
 2 files changed, 92 insertions(+), 1 deletion(-)
 create mode 100644 AllenOnline/tests/refs/mep_passthrough.ref.x86_64_v3-opt

diff --git a/AllenOnline/tests/options/check_duplicates.py b/AllenOnline/tests/options/check_duplicates.py
index 6e6a8b684..d32c99982 100644
--- a/AllenOnline/tests/options/check_duplicates.py
+++ b/AllenOnline/tests/options/check_duplicates.py
@@ -26,7 +26,11 @@ args = parser.parse_args()
 
 rate_lines = []
 add_rate = False
-with open(args.ref[0]) as ref_file:
+
+ref_filename = args.ref[0]
+if os.environ['BINARY_TAG'].startswith('x86_64_v3'):
+    ref_filename += '.x86_64_v3-opt'
+with open(ref_filename) as ref_file:
     for line in ref_file:
         line = line.strip()
         if line.startswith("rate_validator"):
diff --git a/AllenOnline/tests/refs/mep_passthrough.ref.x86_64_v3-opt b/AllenOnline/tests/refs/mep_passthrough.ref.x86_64_v3-opt
new file mode 100644
index 000000000..c3d462fd3
--- /dev/null
+++ b/AllenOnline/tests/refs/mep_passthrough.ref.x86_64_v3-opt
@@ -0,0 +1,87 @@
+DetectorPersistencySvc                 INFO Added successfully Conversion service:XmlCnvSvc
+DetectorDataSvc                     SUCCESS Detector description database: git:/lhcb.xml
+EventClockSvc.FakeEventTime            INFO Event times generated from 0 with steps of 0
+MagneticFieldSvc                       INFO Map scaled by factor 1 with polarity internally used: -1 signed relative current: -1
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M2/Cabling
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M3/Cabling
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M4/Cabling
+DumpMuonGeometry                       INFO Registering /dd/Conditions/ReadoutConf/Muon/Cabling/M5/Cabling
+DumpMuonGeometry                       INFO Number of tiles in tell1 table: 19968
+ToolSvc.PrUTMagnetTool                 INFO Start generation of VeloUT Bdl LUTs
+ToolSvc.PrUTMagnetTool                 INFO Generation of VeloUT Bdl LUTs finished
+ToolSvc.PrUTMagnetTool                 INFO Start generation of VeloUT deflection LUTs
+ToolSvc.PrUTMagnetTool                 INFO Generation of VeloUT deflection LUTs finished
+HLTControlFlowMgr                      INFO Start initialization
+HLTControlFlowMgr                      INFO Will not use an EventSelector.
+HLTControlFlowMgr                      INFO Concurrency level information:
+HLTControlFlowMgr                      INFO  o Number of events slots: 1
+HLTControlFlowMgr                      INFO  o TBB thread pool size:  'ThreadPoolSize':1
+ApplicationMgr                         INFO Application Manager Initialized successfully
+Setting number of slices to 4
+DeFTDetector                           INFO Current FT geometry version =   64
+ApplicationMgr                         INFO Application Manager Started successfully
+DumpFTGeometry                         INFO Conditions DB is compatible with FT bank version 4, 5, 6.
+Starting timer for throughput measurement
+Input complete
+ApplicationMgr                         INFO Application Manager Stopped successfully
+HLTControlFlowMgr                      INFO
+HLTControlFlowMgr                      INFO
+LAZY_AND: allen_non_event_data                #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+ NONLAZY_OR: allen_non_event_data_converters  #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpBeamline/DumpBeamline                   #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpCaloGeometry/DumpCaloGeometry           #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpVPGeometry/DumpVPGeometry               #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpMagneticField/DumpMagneticField         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  DumpFTGeometry/DumpFTGeometry               #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+ NONLAZY_OR: allen_non_event_data_producers   #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer           #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#1         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#2         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#3         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+  AllenTESProducer/AllenTESProducer#4         #=1       Sum=1           Eff=|( 100.0000 +- 0.00000 )%|
+HLTControlFlowMgr                      INFO Histograms converted successfully according to request.
+ToolSvc                                INFO Removing all tools created by ToolSvc
+ApplicationMgr                         INFO Application Manager Finalized successfully
+rate_validator validation:
+Hlt1KsToPiPi_gec:                   26/  2000, (  390.00 +/-    75.99) kHz
+Hlt1TrackMVA_gec:                   22/  2000, (  330.00 +/-    69.97) kHz
+Hlt1TwoTrackMVA_gec:                43/  2000, (  645.00 +/-    97.30) kHz
+Hlt1TwoTrackKs_gec:                  3/  2000, (   45.00 +/-    25.96) kHz
+Hlt1SingleHighPtMuon_gec:            0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1SingleHighPtMuonNoMuID_gec:      1/  2000, (   15.00 +/-    15.00) kHz
+Hlt1LowPtMuon_gec:                 173/  2000, ( 2595.00 +/-   188.57) kHz
+Hlt1D2KK_gec:                        5/  2000, (   75.00 +/-    33.50) kHz
+Hlt1D2KPi_gec:                       8/  2000, (  120.00 +/-    42.34) kHz
+Hlt1D2PiPi_gec:                      5/  2000, (   75.00 +/-    33.50) kHz
+Hlt1DiMuonHighMass_gec:             19/  2000, (  285.00 +/-    65.07) kHz
+Hlt1DiMuonLowMass_gec:              59/  2000, (  885.00 +/-   113.51) kHz
+Hlt1DiMuonSoft_gec:                  0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1LowPtDiMuon_gec:                76/  2000, ( 1140.00 +/-   128.26) kHz
+Hlt1TrackMuonMVA_gec:                0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1TrackElectronMVA_gec:            8/  2000, (  120.00 +/-    42.34) kHz
+Hlt1SingleHighPtElectron_gec:        2/  2000, (   30.00 +/-    21.20) kHz
+Hlt1DisplacedDielectron_gec:         6/  2000, (   90.00 +/-    36.69) kHz
+Hlt1DisplacedLeptons_gec:           14/  2000, (  210.00 +/-    55.93) kHz
+Hlt1SingleHighEt_gec:                0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1Passthrough:                  2000/  2000, (30000.00 +/-     0.00) kHz
+Hlt1Passthrough_gec:              1789/  2000, (26835.00 +/-   206.07) kHz
+Hlt1NoBeam:                          0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BeamOne:                         0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BeamTwo:                         0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BothBeams:                       0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1ODINLumi:                        0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1ODINNoBias:                      0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1VeloMicroBias_gec:               3/  2000, (   45.00 +/-    25.96) kHz
+Hlt1RICH1Alignment_gec:              1/  2000, (   15.00 +/-    15.00) kHz
+Hlt1RICH2Alignment_gec:              0/  2000, (    0.00 +/-     0.00) kHz
+Hlt1BeamGas_gec:                     0/  2000, (    0.00 +/-     0.00) kHz
+Inclusive:                        2000/  2000, (30000.00 +/-     0.00) kHz
+Wrote 2000/2000 events to mep_passthrough.mdf
+ApplicationMgr                         INFO Application Manager Terminated successfully
+Bursts                                 INFO Number of counters : 1
+ |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
+ | "IN"                                            |         1 |
+Events                                 INFO Number of counters : 2
+ |    Counter                                      |     #     |    sum     | mean/eff^* | rms/err^*  |     min     |     max     |
+ | "IN"                                            |      2000 |
+ | "MB_IN"                                         |       679 |
-- 
GitLab


From c599c6bb1f5b32b355caf53f6b6edb277846030c Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Tue, 31 May 2022 21:24:58 +0200
Subject: [PATCH 119/120] Search for zmq also in MooreOnline

---
 AllenOnline/CMakeLists.txt          | 1 +
 cmake/MooreOnlineDependencies.cmake | 4 +++-
 2 files changed, 4 insertions(+), 1 deletion(-)

diff --git a/AllenOnline/CMakeLists.txt b/AllenOnline/CMakeLists.txt
index 50c7c6a6f..ab4cd525f 100644
--- a/AllenOnline/CMakeLists.txt
+++ b/AllenOnline/CMakeLists.txt
@@ -41,6 +41,7 @@ gaudi_add_module(AllenOnlineComp
         LHCb::DAQKernelLib
         LHCb::MDFLib
         LHCb::ZMQLib
+        PkgConfig::zmq
         Online::EventBuildingLib
         Online::GaudiOnline
         Online::OnlineBase
diff --git a/cmake/MooreOnlineDependencies.cmake b/cmake/MooreOnlineDependencies.cmake
index 4d263fdb1..17627580a 100644
--- a/cmake/MooreOnlineDependencies.cmake
+++ b/cmake/MooreOnlineDependencies.cmake
@@ -31,12 +31,14 @@ if(NOT USE_DD4HEP)
   lhcb_find_package(Alignment REQUIRED)
 endif()
 
+find_package(PkgConfig REQUIRED)
+pkg_check_modules(zmq libzmq REQUIRED IMPORTED_TARGET)  # for ZeroMQ
+
 # -- Private dependencies
 if(WITH_MooreOnline_PRIVATE_DEPENDENCIES)
     find_package(Python REQUIRED Interpreter)
 
     find_package(MPI QUIET COMPONENTS C CXX)
 
-    find_package(PkgConfig REQUIRED)
     pkg_check_modules(hwloc QUIET IMPORTED_TARGET hwloc)
 endif()
-- 
GitLab


From 5c6d50742d56e61cb7746078441dfba0d89fc320 Mon Sep 17 00:00:00 2001
From: Roel Aaij <raaij@nikhef.nl>
Date: Wed, 1 Jun 2022 11:05:37 +0200
Subject: [PATCH 120/120] Add zmq and libsodium to public dependencies

---
 cmake/MooreOnlineDependencies.cmake | 1 +
 1 file changed, 1 insertion(+)

diff --git a/cmake/MooreOnlineDependencies.cmake b/cmake/MooreOnlineDependencies.cmake
index 17627580a..769040740 100644
--- a/cmake/MooreOnlineDependencies.cmake
+++ b/cmake/MooreOnlineDependencies.cmake
@@ -33,6 +33,7 @@ endif()
 
 find_package(PkgConfig REQUIRED)
 pkg_check_modules(zmq libzmq REQUIRED IMPORTED_TARGET)  # for ZeroMQ
+pkg_check_modules(sodium libsodium REQUIRED IMPORTED_TARGET)
 
 # -- Private dependencies
 if(WITH_MooreOnline_PRIVATE_DEPENDENCIES)
-- 
GitLab