artdaq  v3_00_01
routing_master.cc
1 #include "MPIProg.hh"
2 #include "artdaq/DAQrate/detail/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 const& 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 const& 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  TestRole_t role_;
139 
140  std::string routing_master_address_;
141  std::string multicast_address_;
142  int token_port_;
143  int table_port_;
144  int ack_port_;
145  std::vector<int> eb_ranks_;
146  int token_count_;
147  size_t token_interval_us_;
148 };
149 
150 RoutingMasterTest::RoutingMasterTest(int argc, char* argv[]) :
151  MPIProg(argc, argv)
152  , pset_(getPset(argc, argv))
153  , daq_pset_(pset_.get<fhicl::ParameterSet>("daq"))
154  , routing_master_address_(daq_pset_.get<std::string>("routing_master_hostname", "localhost"))
155  , multicast_address_(daq_pset_.get<std::string>("table_update_address", "227.128.12.28"))
156  , token_port_(daq_pset_.get<int>("routing_token_port", 35555))
157  , table_port_(daq_pset_.get<int>("table_update_port", 35556))
158  , ack_port_(daq_pset_.get<int>("table_acknowledge_port", 35557))
159  , token_count_(pset_.get<int>("token_count", 1000))
160  , token_interval_us_(pset_.get<size_t>("token_interval_us", 5000))
161 {
162  assert(!(my_rank < 0));
163  switch (my_rank)
164  {
165  case 0:
166  role_ = TestRole_t::TOKEN_GEN;
167  break;
168  case 1:
169  role_ = TestRole_t::ROUTING_MASTER;
170  break;
171  default:
172  role_ = TestRole_t::TABLE_RECEIVER;
173  break;
174  }
175  auto policy_pset = daq_pset_.get<fhicl::ParameterSet>("policy");
176  eb_ranks_ = policy_pset.get<std::vector<int>>("receiver_ranks");
177 
178 }
179 
180 fhicl::ParameterSet RoutingMasterTest::getPset(int argc, char* argv[]) const
181 {
182  std::ostringstream descstr;
183  descstr << "-- <-c <config-file>>";
184  bpo::options_description desc(descstr.str());
185  desc.add_options()
186  ("config,c", bpo::value<std::string>(), "Configuration file.");
187  bpo::variables_map vm;
188  try
189  {
190  bpo::store(bpo::command_line_parser(argc, argv).
191  options(desc).allow_unregistered().run(), vm);
192  bpo::notify(vm);
193  }
194  catch (bpo::error const& e)
195  {
196  std::cerr << "Exception from command line processing in Config::getArtPset: " << e.what() << "\n";
197  throw "cmdline parsing error.";
198  }
199  if (!vm.count("config"))
200  {
201  std::cerr << "Expected \"-- -c <config-file>\" fhicl file specification.\n";
202  throw "cmdline parsing error.";
203  }
204  fhicl::ParameterSet pset;
205  cet::filepath_lookup lookup_policy("FHICL_FILE_PATH");
206  fhicl::make_ParameterSet(vm["config"].as<std::string>(), lookup_policy, pset);
207 
208  return pset;
209 }
210 
212 {
213  if (LockFile::IsLocked("/tmp/routing_master_t.lock")) return;
214  MPI_Barrier(MPI_COMM_WORLD);
215  std::unique_ptr<LockFile> lock;
216  if (my_rank == 0) {
217  lock = std::make_unique<LockFile>("/tmp/routing_master_t.lock");
218  }
219  //std::cout << "daq_pset_: " << daq_pset_.to_string() << std::endl << "conf_.makeParameterSet(): " << conf_.makeParameterSet().to_string() << std::endl;
220 
221  switch (role_)
222  {
223  case TestRole_t::TABLE_RECEIVER:
224  table_receiver();
225  break;
226  case TestRole_t::ROUTING_MASTER:
227  routing_master();
228  break;
229  case TestRole_t::TOKEN_GEN:
230  generate_tokens();
231  break;
232  default:
233  throw "No such node type";
234  }
235  TLOG_DEBUG("routing_master") << "Rank " << my_rank << " complete." << TLOG_ENDL;
236 }
237 
239 {
240  TLOG_DEBUG("generate_tokens") << "Init" << TLOG_ENDL;
241  printHost("generate_tokens");
242  sleep(1);
243 
244  int token_socket = socket(AF_INET, SOCK_STREAM, IPPROTO_TCP);
245  if (token_socket < 0)
246  {
247  TLOG_ERROR("generate_tokens") << "I failed to create the socket for sending Routing Tokens!" << TLOG_ENDL;
248  exit(1);
249  }
250  struct sockaddr_in token_addr;
251  auto sts = ResolveHost(routing_master_address_.c_str(), token_port_, token_addr);
252  if(sts == -1)
253  {
254  TLOG_ERROR("generate_tokens") << "Could not resolve host name" << TLOG_ENDL;
255  }
256 
257  connect(token_socket, (struct sockaddr*)&token_addr, sizeof(token_addr));
258 
259  int sent_tokens = 0;
260  std::map<int, int> token_counter;
261  for(auto rank : eb_ranks_)
262  {
263  token_counter[rank] = 0;
264  }
265  while (sent_tokens < token_count_) {
266  int this_rank = eb_ranks_[seedAndRandom() % eb_ranks_.size()];
267  token_counter[this_rank]++;
269  token.header = TOKEN_MAGIC;
270  token.rank = this_rank;
271  token.new_slots_free = 1;
272 
273  TLOG_DEBUG("generate_tokens") << "Sending RoutingToken " << std::to_string(++sent_tokens) << " for rank " << this_rank << " to " << routing_master_address_ << TLOG_ENDL;
274  send(token_socket, &token, sizeof(artdaq::detail::RoutingToken), 0);
275  usleep(token_interval_us_);
276  }
277  auto max_rank = 0;
278  for(auto rank : token_counter)
279  {
280  if (rank.second > max_rank) max_rank = rank.second;
281  }
282  for(auto rank : token_counter)
283  {
285  token.header = TOKEN_MAGIC;
286  token.rank = rank.first;
287  token.new_slots_free = max_rank - rank.second;
288 
289  TLOG_DEBUG("generate_tokens") << "Sending RoutingToken " << std::to_string(++sent_tokens) << " for rank " << rank.first << " to " << routing_master_address_ << TLOG_ENDL;
290  send(token_socket, &token, sizeof(artdaq::detail::RoutingToken), 0);
291  usleep(token_interval_us_);
292 
293  }
294 
295  TLOG_INFO("generate_tokens") << "Waiting at MPI_Barrier" << TLOG_ENDL;
296  MPI_Barrier(MPI_COMM_WORLD);
297  TLOG_INFO("generate_tokens") << "Done with MPI_Barrier" << TLOG_ENDL;
298 }
299 
301 {
302  TLOG_DEBUG("table_receiver") << "Init" << TLOG_ENDL;
303  printHost("table_receiver");
304 
305 
306  auto table_socket = socket(AF_INET, SOCK_DGRAM, IPPROTO_UDP);
307  if (table_socket < 0)
308  {
309  TLOG_ERROR("table_receiver") << "Error creating socket for receiving data requests!" << TLOG_ENDL;
310  exit(1);
311  }
312 
313  struct sockaddr_in si_me_request;
314 
315  int yes = 1;
316  if (setsockopt(table_socket, SOL_SOCKET, SO_REUSEADDR, &yes, sizeof(yes)) < 0)
317  {
318  TLOG_ERROR("table_receiver") << " Unable to enable port reuse on request socket" << TLOG_ENDL;
319  exit(1);
320  }
321  memset(&si_me_request, 0, sizeof(si_me_request));
322  si_me_request.sin_family = AF_INET;
323  si_me_request.sin_port = htons(table_port_);
324  si_me_request.sin_addr.s_addr = htonl(INADDR_ANY);
325  if (bind(table_socket, (struct sockaddr *)&si_me_request, sizeof(si_me_request)) == -1)
326  {
327  TLOG_ERROR("table_receiver") << "Cannot bind request socket to port " << table_port_ << TLOG_ENDL;
328  exit(1);
329  }
330 
331  struct ip_mreq mreq;
332  long int sts = ResolveHost(multicast_address_.c_str(), mreq.imr_multiaddr);
333  if(sts == -1)
334  {
335  TLOG_ERROR("table_Receiver") << "Unable to resolve multicast hostname" << TLOG_ENDL;
336  exit(1);
337  }
338  mreq.imr_interface.s_addr = htonl(INADDR_ANY);
339  if (setsockopt(table_socket, IPPROTO_IP, IP_ADD_MEMBERSHIP, &mreq, sizeof(mreq)) < 0)
340  {
341  TLOG_ERROR("table_receiver") << "Unable to join multicast group" << TLOG_ENDL;
342  exit(1);
343  }
344 
345  struct epoll_event ev;
346  int table_epoll_fd = epoll_create1(0);
347  ev.events = EPOLLIN | EPOLLPRI;
348  ev.data.fd = table_socket;
349  if (epoll_ctl(table_epoll_fd, EPOLL_CTL_ADD, table_socket, &ev) == -1)
350  {
351  TLOG_ERROR("table_receiver") << "Could not register listen socket to epoll fd" << TLOG_ENDL;
352  exit(3);
353  }
354 
355  auto ack_socket = socket(AF_INET, SOCK_DGRAM, IPPROTO_UDP);
356  struct sockaddr_in ack_addr;
357  sts = ResolveHost(routing_master_address_.c_str(), ack_port_, ack_addr);
358  if(sts == -1)
359  {
360  TLOG_ERROR("table_Receiver") << "Unable to resolve routing master hostname" << TLOG_ENDL;
361  exit(1);
362  }
363 
364  if (table_socket == -1 || table_epoll_fd == -1 || ack_socket == -1)
365  {
366  TLOG_DEBUG("table_receiver") << "One of the listen sockets was not opened successfully." << TLOG_ENDL;
367  exit(4);
368  }
369  artdaq::Fragment::sequence_id_t max_sequence_id = token_count_;
370  artdaq::Fragment::sequence_id_t current_sequence_id = 0;
371  std::map<artdaq::Fragment::sequence_id_t, int> routing_table;
372  TLOG_INFO("table_receiver") << "Expecting " << std::to_string(max_sequence_id) << " as the last Sequence ID in this run" << TLOG_ENDL;
373  while (current_sequence_id < max_sequence_id)
374  {
375  std::vector<epoll_event> table_events_(4);
376  TLOG_DEBUG("table_receiver") << "Waiting for event on table socket" << TLOG_ENDL;
377  auto nfds = epoll_wait(table_epoll_fd, &table_events_[0], table_events_.size(), -1);
378  if (nfds == -1) {
379  perror("epoll_wait");
380  exit(EXIT_FAILURE);
381  }
382 
383  TLOG_DEBUG("table_receiver") << "Received " << nfds << " table update(s)" << TLOG_ENDL;
384  for (auto n = 0; n < nfds; ++n) {
385  auto first = artdaq::Fragment::InvalidSequenceID;
386  auto last = artdaq::Fragment::InvalidSequenceID;
388  recv(table_events_[n].data.fd, &hdr, sizeof(artdaq::detail::RoutingPacketHeader), 0);
389 
390  TLOG_DEBUG("table_receiver") << "Checking for valid header" << TLOG_ENDL;
391  if (hdr.header == ROUTING_MAGIC) {
393  TLOG_DEBUG("table_receiver") << "Receiving data buffer" << TLOG_ENDL;
394  sts = recv(table_events_[n].data.fd, &buffer[0], sizeof(artdaq::detail::RoutingPacketEntry) * hdr.nEntries, 0);
395  assert(static_cast<size_t>(sts) == sizeof(artdaq::detail::RoutingPacketEntry) * hdr.nEntries);
396 
397  first = buffer[0].sequence_id;
398  last = buffer[buffer.size() - 1].sequence_id;
399 
400  for (auto entry : buffer)
401  {
402  if (routing_table.count(entry.sequence_id))
403  {
404  assert(routing_table[entry.sequence_id] == entry.destination_rank);
405  continue;
406  }
407  routing_table[entry.sequence_id] = entry.destination_rank;
408  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;
409  }
410 
412  ack.rank = my_rank;
413  ack.first_sequence_id = first;
414  ack.last_sequence_id = last;
415 
416  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;
417  sendto(ack_socket, &ack, sizeof(artdaq::detail::RoutingAckPacket), 0, (struct sockaddr *)&ack_addr, sizeof(ack_addr));
418  current_sequence_id = last;
419  }
420  }
421  }
422 
423  TLOG_INFO("table_receiver") << "Waiting at MPI_Barrier" << TLOG_ENDL;
424  MPI_Barrier(MPI_COMM_WORLD);
425  TLOG_INFO("table_receiver") << "Done with MPI_Barrier" << TLOG_ENDL;
426 }
427 
429 {
430  TLOG_DEBUG("routing_master") << "Init" << TLOG_ENDL;
431  printHost("routing_master");
432 
433  auto app = std::make_unique<artdaq::RoutingMasterApp>(MPI_COMM_WORLD, "RoutingMaster");
434 
435  app->initialize(pset_, 0, 0);
436  app->do_start(art::RunID(1), 0, 0);
437  TLOG_INFO("routing_master") << "Waiting at MPI_Barrier" << TLOG_ENDL;
438  MPI_Barrier(MPI_COMM_WORLD);
439  TLOG_INFO("routing_master") << "Done with MPI_Barrier, calling RoutingMasterCore::stop" << TLOG_ENDL;
440  app->do_stop(0, 0);
441  TLOG_INFO("routing_master") << "Done with RoutingMasterCore::stop, calling shutdown" << TLOG_ENDL;
442  app->do_shutdown(0);
443  TLOG_INFO("routing_master") << "Done with RoutingMasterCore::shutdown" << TLOG_ENDL;
444 }
445 
446 void RoutingMasterTest::printHost(const std::string& functionName) const
447 {
448  char* doPrint = getenv("PRINT_HOST");
449  if (doPrint == 0) { return; }
450  const int ARRSIZE = 80;
451  char hostname[ARRSIZE];
452  std::string hostString;
453  if (!gethostname(hostname, ARRSIZE))
454  {
455  hostString = hostname;
456  }
457  else
458  {
459  hostString = "unknown";
460  }
461  TLOG_DEBUG("routing_master") << "Running " << functionName
462  << " on host " << hostString
463  << " with rank " << my_rank << "."
464  << TLOG_ENDL;
465 }
466 
467 void printUsage()
468 {
469  int myid = 0;
470  struct rusage usage;
471  getrusage(RUSAGE_SELF, &usage);
472  std::cout << myid << ":"
473  << " user=" << artdaq::TimeUtils::convertUnixTimeToSeconds(usage.ru_utime)
474  << " sys=" << artdaq::TimeUtils::convertUnixTimeToSeconds(usage.ru_stime)
475  << std::endl;
476 }
477 
478 int main(int argc, char* argv[])
479 {
480  artdaq::configureMessageFacility("routing_master", false);
481  int rc = 1;
482  try
483  {
484  RoutingMasterTest p(argc, argv);
485  std::cerr << "Started process " << my_rank << " of " << p.procs_ << ".\n";
486  p.go();
487  rc = 0;
488  }
489  catch (std::string& x)
490  {
491  std::cerr << "Exception (type string) caught in routing_master: "
492  << x
493  << '\n';
494  return 1;
495  }
496  catch (char const* m)
497  {
498  std::cerr << "Exception (type char const*) caught in routing_master: ";
499  if (m)
500  {
501  std::cerr << m;
502  }
503  else
504  {
505  std::cerr << "[the value was a null pointer, so no message is available]";
506  }
507  std::cerr << '\n';
508  }
509  return rc;
510 }
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:27
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 go()
Start the test, using the role assigned.
unsigned new_slots_free
The number of slots free in the token sender (usually 1)
static bool IsLocked(std::string const &path)
Check if the given lock file exists.
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.
Create a &quot;lock file&quot;, removing it upon class destruction.
LockFile(std::string const &path)
Create a lock file with the given path.
uint32_t header
The magic bytes that help validate the RoutingToken.
~LockFile()
LockFile Destructor, removes the lock file.