artdaq  v2_03_02
 All Classes Namespaces Files Functions Variables Typedefs Enumerations Enumerator Pages
routing_master.cc
1 #include "MPIProg.hh"
2 #include "artdaq/Application/Routing/RoutingPacket.hh"
5 #include "cetlib/filepath_maker.h"
6 #include "fhiclcpp/ParameterSet.h"
7 #include "fhiclcpp/make_ParameterSet.h"
8 
9 #include <boost/program_options.hpp>
10 #include <boost/filesystem.hpp>
11 #include "artdaq/Application/RoutingMasterCore.hh"
12 #include "artdaq/Application/RoutingMasterApp.hh"
13 #include <netdb.h>
14 namespace bpo = boost::program_options;
15 
16 #include <algorithm>
17 #include <cmath>
18 #include <cstdio>
19 
20 extern "C"
21 {
22 #include <unistd.h>
23 }
24 
25 #include <iostream>
26 #include <memory>
27 #include <utility>
28 #include <arpa/inet.h>
29 #include <netinet/in.h>
30 #include <sys/types.h>
31 #include <sys/socket.h>
32 
33 extern "C"
34 {
35 #include <sys/time.h>
36 #include <sys/resource.h>
37 }
38 
42 class LockFile
43 {
44 public:
49  explicit LockFile(std::string path) : fileName_(path)
50  {
51  std::ofstream fstream(fileName_);
52  fstream << "Locked" << std::endl;
53  }
54 
59  {
60  if(IsLocked(fileName_)) remove(fileName_.c_str());
61  }
67  static bool IsLocked(std::string path)
68  {
69  return boost::filesystem::exists(path);
70  }
71 
72 private:
73  std::string fileName_;
74 };
75 
79 class RoutingMasterTest : public MPIProg
80 {
81 public:
96  RoutingMasterTest(int argc, char* argv[]);
97 
101  void go();
102 
106  void generate_tokens();
107 
111  void routing_master();
112 
116  void table_receiver();
117 
124  fhicl::ParameterSet getPset(int argc, char* argv[]) const;
125 
126 private:
127  enum class TestRole_t : int
128  {
129  TOKEN_GEN = 0,
130  ROUTING_MASTER = 1,
131  TABLE_RECEIVER = 2
132  };
133 
134  void printHost(const std::string& functionName) const;
135 
136  fhicl::ParameterSet const pset_;
137  fhicl::ParameterSet const daq_pset_;
138  MPI_Comm local_group_comm_;
139  TestRole_t role_;
140 
141  std::string routing_master_address_;
142  std::string multicast_address_;
143  int token_port_;
144  int table_port_;
145  int ack_port_;
146  std::vector<int> eb_ranks_;
147  int token_count_;
148  size_t token_interval_us_;
149 };
150 
151 RoutingMasterTest::RoutingMasterTest(int argc, char* argv[]) :
152  MPIProg(argc, argv)
153  , pset_(getPset(argc, argv))
154  , daq_pset_(pset_.get<fhicl::ParameterSet>("daq"))
155  , local_group_comm_()
156  , routing_master_address_(daq_pset_.get<std::string>("routing_master_hostname", "localhost"))
157  , multicast_address_(daq_pset_.get<std::string>("table_update_address", "227.128.12.28"))
158  , token_port_(daq_pset_.get<int>("routing_token_port", 35555))
159  , table_port_(daq_pset_.get<int>("table_update_port", 35556))
160  , ack_port_(daq_pset_.get<int>("table_acknowledge_port", 35557))
161  , token_count_(pset_.get<int>("token_count", 1000))
162  , token_interval_us_(pset_.get<size_t>("token_interval_us", 5000))
163 {
164  assert(!(my_rank < 0));
165  switch (my_rank)
166  {
167  case 0:
168  role_ = TestRole_t::TOKEN_GEN;
169  break;
170  case 1:
171  role_ = TestRole_t::ROUTING_MASTER;
172  break;
173  default:
174  role_ = TestRole_t::TABLE_RECEIVER;
175  break;
176  }
177  auto policy_pset = daq_pset_.get<fhicl::ParameterSet>("policy");
178  eb_ranks_ = policy_pset.get<std::vector<int>>("receiver_ranks");
179 
180 }
181 
182 fhicl::ParameterSet RoutingMasterTest::getPset(int argc, char* argv[]) const
183 {
184  std::ostringstream descstr;
185  descstr << "-- <-c <config-file>>";
186  bpo::options_description desc(descstr.str());
187  desc.add_options()
188  ("config,c", bpo::value<std::string>(), "Configuration file.");
189  bpo::variables_map vm;
190  try
191  {
192  bpo::store(bpo::command_line_parser(argc, argv).
193  options(desc).allow_unregistered().run(), vm);
194  bpo::notify(vm);
195  }
196  catch (bpo::error const& e)
197  {
198  std::cerr << "Exception from command line processing in Config::getArtPset: " << e.what() << "\n";
199  throw "cmdline parsing error.";
200  }
201  if (!vm.count("config"))
202  {
203  std::cerr << "Expected \"-- -c <config-file>\" fhicl file specification.\n";
204  throw "cmdline parsing error.";
205  }
206  fhicl::ParameterSet pset;
207  cet::filepath_lookup lookup_policy("FHICL_FILE_PATH");
208  fhicl::make_ParameterSet(vm["config"].as<std::string>(), lookup_policy, pset);
209 
210  return pset;
211 }
212 
214 {
215  if (LockFile::IsLocked("/tmp/routing_master_t.lock")) return;
216  MPI_Barrier(MPI_COMM_WORLD);
217  std::unique_ptr<LockFile> lock;
218  if (my_rank == 0) {
219  lock = std::make_unique<LockFile>("/tmp/routing_master_t.lock");
220  }
221  //std::cout << "daq_pset_: " << daq_pset_.to_string() << std::endl << "conf_.makeParameterSet(): " << conf_.makeParameterSet().to_string() << std::endl;
222  MPI_Comm_split(MPI_COMM_WORLD, static_cast<int>(role_), 0, &local_group_comm_);
223  switch (role_)
224  {
225  case TestRole_t::TABLE_RECEIVER:
226  table_receiver();
227  break;
228  case TestRole_t::ROUTING_MASTER:
229  routing_master();
230  break;
231  case TestRole_t::TOKEN_GEN:
232  generate_tokens();
233  break;
234  default:
235  throw "No such node type";
236  }
237  TLOG_DEBUG("routing_master") << "Rank " << my_rank << " complete." << TLOG_ENDL;
238 }
239 
241 {
242  TLOG_DEBUG("generate_tokens") << "Init" << TLOG_ENDL;
243  printHost("generate_tokens");
244  sleep(1);
245 
246  int token_socket = socket(AF_INET, SOCK_STREAM, IPPROTO_TCP);
247  if (!token_socket)
248  {
249  TLOG_ERROR("generate_tokens") << "I failed to create the socket for sending Routing Tokens!" << TLOG_ENDL;
250  exit(1);
251  }
252  struct sockaddr_in token_addr;
253  auto sts = ResolveHost(routing_master_address_.c_str(), token_port_, token_addr);
254  if(sts == -1)
255  {
256  TLOG_ERROR("generate_tokens") << "Could not resolve host name" << TLOG_ENDL;
257  }
258 
259  connect(token_socket, (struct sockaddr*)&token_addr, sizeof(token_addr));
260 
261  int sent_tokens = 0;
262  std::map<int, int> token_counter;
263  for(auto rank : eb_ranks_)
264  {
265  token_counter[rank] = 0;
266  }
267  while (sent_tokens < token_count_) {
268  int this_rank = eb_ranks_[rand() % eb_ranks_.size()];
269  token_counter[this_rank]++;
271  token.header = TOKEN_MAGIC;
272  token.rank = this_rank;
273  token.new_slots_free = 1;
274 
275  TLOG_DEBUG("generate_tokens") << "Sending RoutingToken " << std::to_string(++sent_tokens) << " for rank " << this_rank << " to " << routing_master_address_ << TLOG_ENDL;
276  send(token_socket, &token, sizeof(artdaq::detail::RoutingToken), 0);
277  usleep(token_interval_us_);
278  }
279  auto max_rank = 0;
280  for(auto rank : token_counter)
281  {
282  if (rank.second > max_rank) max_rank = rank.second;
283  }
284  for(auto rank : token_counter)
285  {
287  token.header = TOKEN_MAGIC;
288  token.rank = rank.first;
289  token.new_slots_free = max_rank - rank.second;
290 
291  TLOG_DEBUG("generate_tokens") << "Sending RoutingToken " << std::to_string(++sent_tokens) << " for rank " << rank.first << " to " << routing_master_address_ << TLOG_ENDL;
292  send(token_socket, &token, sizeof(artdaq::detail::RoutingToken), 0);
293  usleep(token_interval_us_);
294 
295  }
296 
297  MPI_Comm_free(&local_group_comm_);
298  TLOG_INFO("generate_tokens") << "Waiting at MPI_Barrier" << TLOG_ENDL;
299  MPI_Barrier(MPI_COMM_WORLD);
300  TLOG_INFO("generate_tokens") << "Done with MPI_Barrier" << TLOG_ENDL;
301 }
302 
304 {
305  TLOG_DEBUG("table_receiver") << "Init" << TLOG_ENDL;
306  printHost("table_receiver");
307 
308 
309  auto table_socket = socket(AF_INET, SOCK_DGRAM, IPPROTO_UDP);
310  if (!table_socket)
311  {
312  TLOG_ERROR("table_receiver") << "Error creating socket for receiving data requests!" << TLOG_ENDL;
313  exit(1);
314  }
315 
316  struct sockaddr_in si_me_request;
317 
318  int yes = 1;
319  if (setsockopt(table_socket, SOL_SOCKET, SO_REUSEADDR, &yes, sizeof(yes)) < 0)
320  {
321  TLOG_ERROR("table_receiver") << " Unable to enable port reuse on request socket" << TLOG_ENDL;
322  exit(1);
323  }
324  memset(&si_me_request, 0, sizeof(si_me_request));
325  si_me_request.sin_family = AF_INET;
326  si_me_request.sin_port = htons(table_port_);
327  si_me_request.sin_addr.s_addr = htonl(INADDR_ANY);
328  if (bind(table_socket, (struct sockaddr *)&si_me_request, sizeof(si_me_request)) == -1)
329  {
330  TLOG_ERROR("table_receiver") << "Cannot bind request socket to port " << table_port_ << TLOG_ENDL;
331  exit(1);
332  }
333 
334  struct ip_mreq mreq;
335  long int sts = ResolveHost(multicast_address_.c_str(), mreq.imr_multiaddr);
336  if(sts == -1)
337  {
338  TLOG_ERROR("table_Receiver") << "Unable to resolve multicast hostname" << TLOG_ENDL;
339  exit(1);
340  }
341  mreq.imr_interface.s_addr = htonl(INADDR_ANY);
342  if (setsockopt(table_socket, IPPROTO_IP, IP_ADD_MEMBERSHIP, &mreq, sizeof(mreq)) < 0)
343  {
344  TLOG_ERROR("table_receiver") << "Unable to join multicast group" << TLOG_ENDL;
345  exit(1);
346  }
347 
348  struct epoll_event ev;
349  int table_epoll_fd = epoll_create1(0);
350  ev.events = EPOLLIN | EPOLLPRI;
351  ev.data.fd = table_socket;
352  if (epoll_ctl(table_epoll_fd, EPOLL_CTL_ADD, table_socket, &ev) == -1)
353  {
354  TLOG_ERROR("table_receiver") << "Could not register listen socket to epoll fd" << TLOG_ENDL;
355  exit(3);
356  }
357 
358  auto ack_socket = socket(AF_INET, SOCK_DGRAM, IPPROTO_UDP);
359  struct sockaddr_in ack_addr;
360  sts = ResolveHost(routing_master_address_.c_str(), ack_port_, ack_addr);
361  if(sts == -1)
362  {
363  TLOG_ERROR("table_Receiver") << "Unable to resolve routing master hostname" << TLOG_ENDL;
364  exit(1);
365  }
366 
367  if (table_socket == -1 || table_epoll_fd == -1 || ack_socket == -1)
368  {
369  TLOG_DEBUG("table_receiver") << "One of the listen sockets was not opened successfully." << TLOG_ENDL;
370  exit(4);
371  }
372  artdaq::Fragment::sequence_id_t max_sequence_id = token_count_;
373  artdaq::Fragment::sequence_id_t current_sequence_id = 0;
374  std::map<artdaq::Fragment::sequence_id_t, int> routing_table;
375  TLOG_INFO("table_receiver") << "Expecting " << std::to_string(max_sequence_id) << " as the last Sequence ID in this run" << TLOG_ENDL;
376  while (current_sequence_id < max_sequence_id)
377  {
378  std::vector<epoll_event> table_events_(4);
379  TLOG_DEBUG("table_receiver") << "Waiting for event on table socket" << TLOG_ENDL;
380  auto nfds = epoll_wait(table_epoll_fd, &table_events_[0], table_events_.size(), -1);
381  if (nfds == -1) {
382  perror("epoll_wait");
383  exit(EXIT_FAILURE);
384  }
385 
386  TLOG_DEBUG("table_receiver") << "Received " << nfds << " table update(s)" << TLOG_ENDL;
387  for (auto n = 0; n < nfds; ++n) {
388  auto first = artdaq::Fragment::InvalidSequenceID;
389  auto last = artdaq::Fragment::InvalidSequenceID;
391  recv(table_events_[n].data.fd, &hdr, sizeof(artdaq::detail::RoutingPacketHeader), 0);
392 
393  TLOG_DEBUG("table_receiver") << "Checking for valid header" << TLOG_ENDL;
394  if (hdr.header == ROUTING_MAGIC) {
396  TLOG_DEBUG("table_receiver") << "Receiving data buffer" << TLOG_ENDL;
397  sts = recv(table_events_[n].data.fd, &buffer[0], sizeof(artdaq::detail::RoutingPacketEntry) * hdr.nEntries, 0);
398  assert(static_cast<size_t>(sts) == sizeof(artdaq::detail::RoutingPacketEntry) * hdr.nEntries);
399 
400  first = buffer[0].sequence_id;
401  last = buffer[buffer.size() - 1].sequence_id;
402 
403  for (auto entry : buffer)
404  {
405  if (routing_table.count(entry.sequence_id))
406  {
407  assert(routing_table[entry.sequence_id] == entry.destination_rank);
408  continue;
409  }
410  routing_table[entry.sequence_id] = entry.destination_rank;
411  TLOG_DEBUG("table_receiver") << "table_receiver " << std::to_string(my_rank) << ": received update: SeqID " << std::to_string(entry.sequence_id) << " -> Rank " << std::to_string(entry.destination_rank) << TLOG_ENDL;
412  }
413 
415  ack.rank = my_rank;
416  ack.first_sequence_id = first;
417  ack.last_sequence_id = last;
418 
419  TLOG_DEBUG("table_receiver") << "Sending RoutingAckPacket with first= " << std::to_string(first) << " and last= " << std::to_string(last) << " to " << routing_master_address_ << ", port " << ack_port_ << TLOG_ENDL
420  sendto(ack_socket, &ack, sizeof(artdaq::detail::RoutingAckPacket), 0, (struct sockaddr *)&ack_addr, sizeof(ack_addr));
421  current_sequence_id = last;
422  }
423  }
424  }
425 
426  MPI_Comm_free(&local_group_comm_);
427  TLOG_INFO("table_receiver") << "Waiting at MPI_Barrier" << TLOG_ENDL;
428  MPI_Barrier(MPI_COMM_WORLD);
429  TLOG_INFO("table_receiver") << "Done with MPI_Barrier" << TLOG_ENDL;
430 }
431 
433 {
434  TLOG_DEBUG("routing_master") << "Init" << TLOG_ENDL;
435  printHost("routing_master");
436 
437  auto app = std::make_unique<artdaq::RoutingMasterApp>(local_group_comm_, "RoutingMaster");
438 
439  app->initialize(pset_, 0, 0);
440  app->do_start(art::RunID(1), 0, 0);
441  TLOG_INFO("routing_master") << "Waiting at MPI_Barrier" << TLOG_ENDL;
442  MPI_Barrier(MPI_COMM_WORLD);
443  TLOG_INFO("routing_master") << "Done with MPI_Barrier, calling RoutingMasterCore::stop" << TLOG_ENDL;
444  app->do_stop(0, 0);
445  TLOG_INFO("routing_master") << "Done with RoutingMasterCore::stop, calling shutdown" << TLOG_ENDL;
446  app->do_shutdown(0);
447  TLOG_INFO("routing_master") << "Done with RoutingMasterCore::shutdown" << TLOG_ENDL;
448  MPI_Comm_free(&local_group_comm_);
449 }
450 
451 void RoutingMasterTest::printHost(const std::string& functionName) const
452 {
453  char* doPrint = getenv("PRINT_HOST");
454  if (doPrint == 0) { return; }
455  const int ARRSIZE = 80;
456  char hostname[ARRSIZE];
457  std::string hostString;
458  if (!gethostname(hostname, ARRSIZE))
459  {
460  hostString = hostname;
461  }
462  else
463  {
464  hostString = "unknown";
465  }
466  TLOG_DEBUG("routing_master") << "Running " << functionName
467  << " on host " << hostString
468  << " with rank " << my_rank << "."
469  << TLOG_ENDL;
470 }
471 
472 void printUsage()
473 {
474  int myid = 0;
475  struct rusage usage;
476  getrusage(RUSAGE_SELF, &usage);
477  std::cout << myid << ":"
478  << " user=" << artdaq::Globals::timevalAsDouble(usage.ru_utime)
479  << " sys=" << artdaq::Globals::timevalAsDouble(usage.ru_stime)
480  << std::endl;
481 }
482 
483 int main(int argc, char* argv[])
484 {
485  artdaq::configureMessageFacility("routing_master", false);
486  int rc = 1;
487  try
488  {
489  RoutingMasterTest p(argc, argv);
490  std::cerr << "Started process " << my_rank << " of " << p.procs_ << ".\n";
491  p.go();
492  rc = 0;
493  }
494  catch (std::string& x)
495  {
496  std::cerr << "Exception (type string) caught in routing_master: "
497  << x
498  << '\n';
499  return 1;
500  }
501  catch (char const* m)
502  {
503  std::cerr << "Exception (type char const*) caught in routing_master: ";
504  if (m)
505  {
506  std::cerr << m;
507  }
508  else
509  {
510  std::cerr << "[the value was a null pointer, so no message is available]";
511  }
512  std::cerr << '\n';
513  }
514  return rc;
515 }
The RoutingMasterTest class runs the routing_master test.
int ResolveHost(char const *host_in, in_addr &addr)
Convert a string hostname to a in_addr suitable for socket communication.
Definition: TCPConnect.cc:29
A row of the Routing Table.
void routing_master()
Load a RoutingMasterCore instance, receive tokens from the token generators, and send table updates t...
A wrapper for a MPI program. Similar to MPISentry.
Definition: MPIProg.hh:10
A RoutingAckPacket contains the rank of the table receiver, plus the first and last sequence IDs in t...
void configureMessageFacility(char const *progname, bool useConsole=true)
Configure and start the message facility. Provide the program name so that messages will be appropria...
void go()
Start the test, using the role assigned.
unsigned new_slots_free
The number of slots free in the token sender (usually 1)
The RoutingToken contains the magic bytes, the rank of the token sender, and the number of slots free...
RoutingMasterTest(int argc, char *argv[])
RoutingMasterTest Constructor.
void table_receiver()
Receive Routing Tables from the Routing Master and send acknowledgement packets back.
Fragment::sequence_id_t first_sequence_id
The first sequence ID in the received RoutingPacket.
fhicl::ParameterSet getPset(int argc, char *argv[]) const
Parse the command line arguments and load a configuration FHiCL file.
The header of the Routing Table, containing the magic bytes and the number of entries.
uint32_t header
Magic bytes to make sure the packet wasn&#39;t garbled.
Fragment::sequence_id_t last_sequence_id
The last sequence ID in the received RoutingPacket.
std::vector< RoutingPacketEntry > RoutingPacket
A RoutingPacket is simply a vector of RoutingPacketEntry objects. It is not suitable for network tran...
int rank
The rank from which the RoutingToken came.
int rank
The rank from which the RoutingAckPacket came.
size_t nEntries
The number of RoutingPacketEntries in the RoutingPacket.
void generate_tokens()
Generate tokens and send them to the Routing Master.
static bool IsLocked(std::string path)
Check if the given lock file exists.
Create a &quot;lock file&quot;, removing it upon class destruction.
LockFile(std::string path)
Create a lock file with the given path.
static double timevalAsDouble(struct timeval tv)
Convert a timeval value to a double.
Definition: Globals.cc:6
uint32_t header
The magic bytes that help validate the RoutingToken.
~LockFile()
LockFile Destructor, removes the lock file.