artdaq  v2_03_02
 All Classes Namespaces Files Functions Variables Typedefs Enumerations Enumerator Pages
EventStore.cc
1 #include "artdaq/DAQdata/Globals.hh" // Before trace.h gets included in ConcurrentQueue (from GlobalQueue)
2 #include "artdaq/DAQrate/EventStore.hh"
3 #include <utility>
4 #include <cstring>
5 #include <dlfcn.h>
6 #include <iomanip>
7 #include <fstream>
8 #include <sstream>
9 #include <thread>
10 #include <chrono>
11 
12 #include "cetlib/exception.h"
13 #include "artdaq-core/Core/StatisticsCollection.hh"
14 #include "artdaq-core/Core/SimpleQueueReader.hh"
15 #include "artdaq/Application/Routing/RoutingPacket.hh"
17 
18 using namespace std;
19 
20 namespace artdaq
21 {
22  const std::string EventStore::EVENT_RATE_STAT_KEY("EventStoreEventRate");
23  const std::string EventStore::INCOMPLETE_EVENT_STAT_KEY("EventStoreIncompleteEvents");
24 
25  EventStore::EventStore(const fhicl::ParameterSet& pset, size_t num_fragments_per_event, run_id_t run,
26  size_t event_queue_depth, size_t max_incomplete_event_count)
27  : num_fragments_per_event_(num_fragments_per_event)
28  , max_queue_size_(pset.get<size_t>("event_queue_depth", event_queue_depth))
29  , max_incomplete_count_(pset.get<size_t>("max_incomplete_events", max_incomplete_event_count))
30  , run_id_(run)
31  , subrun_id_(0)
32  , events_()
33  , queue_(getGlobalQueue(max_queue_size_))
34  , reader_thread_launch_time_(std::chrono::steady_clock::now())
35  , send_requests_(pset.get<bool>("send_requests", false))
36  , active_requests_()
37  , request_port_(pset.get<int>("request_port", 3001))
38  , request_delay_(pset.get<size_t>("request_delay_ms", 10))
39  , multicast_out_addr_(pset.get<std::string>("output_address", "localhost"))
40  , request_mode_(detail::RequestMessageMode::Normal)
41  , seqIDModulus_(1)
42  , lastFlushedSeqID_(0)
43  , highestSeqIDSeen_(0)
44  , enq_timeout_(pset.get<double>("event_queue_wait_time", 5.0))
45  , enq_check_count_(pset.get<size_t>("event_queue_check_count", 5000))
46  , printSummaryStats_(pset.get<bool>("print_event_store_stats", false))
47  , incomplete_event_report_interval_ms_(pset.get<int>("incomplete_event_report_interval_ms", -1))
48  , last_incomplete_event_report_time_(std::chrono::steady_clock::now())
49  , token_socket_(-1)
50  , art_thread_wait_ms_(pset.get<int>("art_thread_wait_ms", 4000))
51  {
52  TLOG_DEBUG("EventStore") << "EventStore CONSTRUCTOR" << TLOG_ENDL;
53  initStatistics_();
54  setup_requests_(pset.get<std::string>("request_address", "227.128.12.26"));
55 
56  auto rmConfig = pset.get<fhicl::ParameterSet>("routing_token_config", fhicl::ParameterSet());
57  send_routing_tokens_ = rmConfig.get<bool>("use_routing_master", false);
58  token_port_ = rmConfig.get<int>("routing_token_port", 35555);
59  token_address_ = rmConfig.get<std::string>("routing_master_hostname", "localhost");
60  setup_tokens_();
61  TRACE(12, "artdaq::EventStore::EventStore ctor - reader_thread_ initialized");
62  }
63 
64  EventStore::EventStore(const fhicl::ParameterSet& pset,
65  size_t num_fragments_per_event,
66  run_id_t run,
67  int argc,
68  char* argv[],
69  ART_CMDLINE_FCN* reader)
70  : EventStore(pset, num_fragments_per_event, run, 50, 50)
71  {
72  reader_thread_ = (std::async(std::launch::async, reader, argc, argv));
73  }
74 
75  EventStore::EventStore(const fhicl::ParameterSet& pset,
76  size_t num_fragments_per_event,
77  run_id_t run,
78  const std::string& configString,
79  ART_CFGSTRING_FCN* reader)
80  : EventStore(pset, num_fragments_per_event, run, 20, 20)
81  {
82  reader_thread_ = (std::async(std::launch::async, reader, configString));
83  }
84 
86  {
87  TLOG_DEBUG("EventStore") << "Shutting down EventStore" << TLOG_ENDL;
88  if (printSummaryStats_)
89  {
90  reportStatistics_();
91  }
92  shutdown(request_socket_, 2);
93  close(request_socket_);
94  shutdown(token_socket_, 2);
95  close(token_socket_);
96  }
97 
98  void EventStore::insert(FragmentPtr pfrag,
99  bool printWarningWhenFragmentIsDropped)
100  {
101  // We should never get a null pointer, nor should we get a
102  // Fragment without a good fragment ID.
103  assert(pfrag != nullptr);
104  assert(pfrag->fragmentID() != Fragment::InvalidFragmentID);
105 
106  // find the event being built and put the fragment into it,
107  // start new event if not already present
108  // if the event is complete, delete it and report timing
109 
110  // The sequenceID is expected to be correct in the incoming fragment.
111  // The EventStore will divide it by the seqIDModulus to support the use case
112  // of the aggregator which needs to bunch groups of serialized events with
113  // continuous sequence IDs together.
114  if (pfrag->sequenceID() > highestSeqIDSeen_)
115  {
116  highestSeqIDSeen_ = pfrag->sequenceID();
117 
118  // Get the timestamp of this fragment, in experiment-defined clocks
119  Fragment::timestamp_t timestamp = pfrag->timestamp();
120 
121  // Send a request to the board readers!
122  if (send_requests_)
123  {
124  std::lock_guard<std::mutex> lk(request_mutex_);
125  active_requests_[highestSeqIDSeen_] = timestamp;
126  send_request_();
127  }
128  }
129 
130  // When we're in the "EndOfRun" condition, send requests for EVERY fragment inserted!
131  // This helps to make sure that all BoardReaders get the EndOfRun request and send all their data.
132  if (send_requests_ && request_mode_ == detail::RequestMessageMode::EndOfRun)
133  {
134  std::lock_guard<std::mutex> lk(request_mutex_);
135  send_request_();
136  }
137  Fragment::sequence_id_t sequence_id = ((pfrag->sequenceID() - (1 + lastFlushedSeqID_)) / seqIDModulus_) + 1;
138  TRACE(13, "EventStore::insert seq=%lu fragID=%d id=%d lastFlushed=%lu seqIDMod=%d seq=%lu"
139  , pfrag->sequenceID(), pfrag->fragmentID(), my_rank, lastFlushedSeqID_, seqIDModulus_, sequence_id);
140 
141 
142  // Find if the right event id is already known to events_ and, if so, where
143  // it is.
144  EventMap::iterator loc = events_.lower_bound(sequence_id);
145 
146  if (loc == events_.end() || events_.key_comp()(sequence_id, loc->first))
147  {
148  // We don't have an event with this id; create one and insert it at loc,
149  // and ajust loc to point to the newly inserted event.
150  RawEvent_ptr newevent(new RawEvent(run_id_, subrun_id_, pfrag->sequenceID()));
151  loc =
152  events_.insert(loc, EventMap::value_type(sequence_id, newevent));
153  }
154 
155  // Now insert the fragment into the event we have located.
156  loc->second->insertFragment(std::move(pfrag));
157  if (loc->second->numFragments() == num_fragments_per_event_)
158  {
159  // This RawEvent is complete; capture it, remove it from the
160  // map, report on statistics, and put the shared pointer onto
161  // the event queue.
162  RawEvent_ptr complete_event(loc->second);
163  complete_event->markComplete();
164 
165  events_.erase(loc);
166 
167  if (send_requests_)
168  {
169  std::lock_guard<std::mutex> lk(request_mutex_);
170  active_requests_.erase(sequence_id);
171  }
172  // 13-Dec-2012, KAB - this monitoring needs to come before
173  // the enqueueing of the event lest it be empty by the
174  // time that we ask for the word count.
175  MonitoredQuantityPtr mqPtr = StatisticsCollection::getInstance().
176  getMonitoredQuantity(EVENT_RATE_STAT_KEY);
177  if (mqPtr.get() != 0)
178  {
179  mqPtr->addSample(complete_event->wordCount());
180  }
181  TRACE(14, "EventStore::insert seq=%lu enqTimedWait start", sequence_id);
182  bool enqSuccess = queue_.enqTimedWait(complete_event, enq_timeout_);
183  TRACE(enqSuccess ? 14 : 0, "EventStore::insert seq=%lu enqTimedWait complete", sequence_id);
184  if (!enqSuccess)
185  {
186  //TRACE_CNTL( "modeM", 0 );
187  if (printWarningWhenFragmentIsDropped)
188  {
189  TLOG_WARNING("EventStore") << "Enqueueing event " << sequence_id
190  << " FAILED, queue size = "
191  << queue_.size() <<
192  "; apparently no events were removed from this process's queue during the " << std::to_string(enq_timeout_.count())
193  << "-second timeout period" << TLOG_ENDL;
194  }
195  else
196  {
197  TLOG_DEBUG("EventStore") << "Enqueueing event " << sequence_id
198  << " FAILED, queue size = "
199  << queue_.size() <<
200  "; apparently no events were removed from this process's queue during the " << std::to_string(enq_timeout_.count())
201  << "-second timeout period" << TLOG_ENDL;
202  }
203  }
204  else
205  {
206  send_routing_token_(1);
207  }
208  }
209  MonitoredQuantityPtr mqPtr = StatisticsCollection::getInstance().
210  getMonitoredQuantity(INCOMPLETE_EVENT_STAT_KEY);
211  if (mqPtr.get() != 0)
212  {
213  mqPtr->addSample(events_.size());
214  }
215  }
216 
217  EventStore::EventStoreInsertResult EventStore::insert(FragmentPtr pfrag, FragmentPtr& rejectedFragment)
218  {
219  // Test whether this fragment can be safely accepted. If we accept
220  // it, and it completes an event, then we want to be sure that it
221  // can be pushed onto the event queue. If not, we return it and
222  // let the caller know that we didn't accept it.
223  TRACE(12, "EventStore: Testing if queue is full");
224  if (queue_.full())
225  {
226  size_t sleepTime = 1000000 * (enq_timeout_.count() / enq_check_count_);
227  TRACE(12, "EventStore: sleepTime is %lu.", sleepTime);
228  size_t loopCount = 0;
229  while (loopCount < enq_check_count_ && queue_.full())
230  {
231  ++loopCount;
232  usleep(sleepTime);
233  }
234  if (queue_.full())
235  {
236  rejectedFragment = std::move(pfrag);
238  }
239  }
240  TRACE(12, "EventStore: Testing if there's room in the EventStore");
241  auto incomplete_full = events_.size() >= max_incomplete_count_;
242  if (incomplete_full)
243  {
244  EventMap::iterator loc = events_.lower_bound(pfrag->sequenceID());
245 
246  if (loc == events_.end() || events_.key_comp()(pfrag->sequenceID(), loc->first))
247  {
248  rejectedFragment = std::move(pfrag);
250  }
251  }
252 
253  TRACE(12, "EventStore: Performing insert");
254  insert(std::move(pfrag));
256  }
257 
258  bool
259  EventStore::endOfData(int& readerReturnValue)
260  {
261  TLOG_DEBUG("EventStore") << "EventStore::endOfData" << TLOG_ENDL;
262  RawEvent_ptr end_of_data(nullptr);
263  TRACE(4, "EventStore::endOfData: Enqueuing end_of_data event");
264  bool enqSuccess = queue_.enqTimedWait(end_of_data, enq_timeout_);
265  if (!enqSuccess)
266  {
267  return false;
268  }
269  TRACE(4, "EventStore::endOfData: Getting return code from art thread");
270  readerReturnValue = reader_thread_.get();
271  return true;
272  }
273 
274  void EventStore::setSeqIDModulus(unsigned int seqIDModulus)
275  {
276  seqIDModulus_ = seqIDModulus;
277  }
278 
280  {
281  bool enqSuccess;
282  size_t initialStoreSize = events_.size();
283  TLOG_DEBUG("EventStore") << "Flushing " << initialStoreSize
284  << " stale events from the EventStore." << TLOG_ENDL;
285  EventMap::iterator loc;
286  std::vector<sequence_id_t> flushList;
287  for (loc = events_.begin(); loc != events_.end(); ++loc)
288  {
289  RawEvent_ptr complete_event(loc->second);
290  MonitoredQuantityPtr mqPtr = StatisticsCollection::getInstance().
291  getMonitoredQuantity(EVENT_RATE_STAT_KEY);
292  if (mqPtr.get() != 0)
293  {
294  mqPtr->addSample(complete_event->wordCount());
295  }
296  enqSuccess = queue_.enqTimedWait(complete_event, enq_timeout_);
297  if (!enqSuccess)
298  {
299  break;
300  }
301  else
302  {
303  flushList.push_back(loc->first);
304  }
305  }
306  for (size_t idx = 0; idx < flushList.size(); ++idx)
307  {
308  events_.erase(flushList[idx]);
309  }
310  TLOG_DEBUG("EventStore") << "Done flushing " << flushList.size()
311  << " stale events from the EventStore." << TLOG_ENDL;
312 
313  lastFlushedSeqID_ = highestSeqIDSeen_;
314  return (flushList.size() >= initialStoreSize);
315  }
316 
318  {
319  if (!queue_.queueReaderIsReady())
320  {
321  TLOG_WARNING("EventStore") << "Run start requested, but the art thread is not yet ready, waiting up to " << art_thread_wait_ms_ << " msec..." << TLOG_ENDL;
322  while (!queue_.queueReaderIsReady() && std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::steady_clock::now() - reader_thread_launch_time_).count() < art_thread_wait_ms_)
323  {
324  usleep(1000); // wait 1 ms
325  }
326  if (queue_.queueReaderIsReady())
327  {
328  auto dur = std::chrono::duration_cast<std::chrono::milliseconds>(queue_.getReadyTime() - reader_thread_launch_time_).count();
329  TLOG_INFO("EventStore") << "art initialization took (roughly) " << std::setw(4) << std::to_string(dur) << " ms." << TLOG_ENDL;
330  }
331  else
332  {
333  auto dur = std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::steady_clock::now() - reader_thread_launch_time_).count();
334  TLOG_ERROR("EventStore") << "art thread still not ready after " << dur << " ms. Continuing to start..." << TLOG_ENDL;
335  }
336  }
337  run_id_ = runID;
338  subrun_id_ = 1;
339  lastFlushedSeqID_ = 0;
340  highestSeqIDSeen_ = 0;
341  send_routing_token_(max_queue_size_);
342  TLOG_DEBUG("EventStore") << "Starting run " << run_id_
343  << ", max queue size = "
344  << max_queue_size_
345  << ", queue capacity = "
346  << queue_.capacity()
347  << ", queue size = "
348  << queue_.size() << TLOG_ENDL;
349  if (metricMan)
350  {
351  double runSubrun = run_id_ + ((double)subrun_id_ / 10000);
352  metricMan->sendMetric("Run Number", runSubrun, "Run:Subrun", 1, false);
353  }
354  }
355 
357  {
358  ++subrun_id_;
359  if (metricMan)
360  {
361  double runSubrun = run_id_ + ((double)subrun_id_ / 10000);
362  metricMan->sendMetric("Run Number", runSubrun, "Run:Subrun", 1, false);
363  }
364  }
365 
367  {
368  RawEvent_ptr endOfRunEvent(new RawEvent(run_id_, subrun_id_, 0));
369  std::unique_ptr<artdaq::Fragment>
370  endOfRunFrag(new
371  Fragment(static_cast<size_t>
372  (ceil(sizeof(my_rank) /
373  static_cast<double>(sizeof(Fragment::value_type))))));
374 
375  endOfRunFrag->setSystemType(Fragment::EndOfRunFragmentType);
376  *endOfRunFrag->dataBegin() = my_rank;
377  endOfRunEvent->insertFragment(std::move(endOfRunFrag));
378 
379  return queue_.enqTimedWait(endOfRunEvent, enq_timeout_);
380  }
381 
383  {
384  RawEvent_ptr endOfSubrunEvent(new RawEvent(run_id_, subrun_id_, 0));
385  std::unique_ptr<artdaq::Fragment>
386  endOfSubrunFrag(new
387  Fragment(static_cast<size_t>
388  (ceil(sizeof(my_rank) /
389  static_cast<double>(sizeof(Fragment::value_type))))));
390 
391  endOfSubrunFrag->setSystemType(Fragment::EndOfSubrunFragmentType);
392  *endOfSubrunFrag->dataBegin() = my_rank;
393  endOfSubrunEvent->insertFragment(std::move(endOfSubrunFrag));
394 
395  return queue_.enqTimedWait(endOfSubrunEvent, enq_timeout_);
396  }
397 
398  void
399  EventStore::initStatistics_()
400  {
401  MonitoredQuantityPtr mqPtr = StatisticsCollection::getInstance().
402  getMonitoredQuantity(EVENT_RATE_STAT_KEY);
403  if (mqPtr.get() == 0)
404  {
405  mqPtr.reset(new MonitoredQuantity(3.0, 300.0));
406  StatisticsCollection::getInstance().
407  addMonitoredQuantity(EVENT_RATE_STAT_KEY, mqPtr);
408  }
409  mqPtr->reset();
410 
411  mqPtr = StatisticsCollection::getInstance().
412  getMonitoredQuantity(INCOMPLETE_EVENT_STAT_KEY);
413  if (mqPtr.get() == 0)
414  {
415  mqPtr.reset(new MonitoredQuantity(3.0, 300.0));
416  StatisticsCollection::getInstance().
417  addMonitoredQuantity(INCOMPLETE_EVENT_STAT_KEY, mqPtr);
418  }
419  mqPtr->reset();
420  }
421 
422  void
423  EventStore::reportStatistics_()
424  {
425  MonitoredQuantityPtr mqPtr = StatisticsCollection::getInstance().
426  getMonitoredQuantity(EVENT_RATE_STAT_KEY);
427  if (mqPtr.get() != 0)
428  {
429  ostringstream oss;
430  oss << EVENT_RATE_STAT_KEY << "_" << setfill('0') << setw(4) << run_id_
431  << "_" << setfill('0') << setw(4) << my_rank << ".txt";
432  std::string filename = oss.str();
433  ofstream outStream(filename.c_str());
434  mqPtr->waitUntilAccumulatorsHaveBeenFlushed(3.0);
435  artdaq::MonitoredQuantityStats stats;
436  mqPtr->getStats(stats);
437  outStream << "EventStore rank " << my_rank << ": events processed = "
438  << stats.fullSampleCount << " at " << stats.fullSampleRate
439  << " events/sec, data rate = "
440  << (stats.fullValueRate * sizeof(RawDataType)
441  / 1024.0 / 1024.0) << " MB/sec, duration = "
442  << stats.fullDuration << " sec" << std::endl
443  << " minimum event size = "
444  << (stats.fullValueMin * sizeof(RawDataType)
445  / 1024.0 / 1024.0)
446  << " MB, maximum event size = "
447  << (stats.fullValueMax * sizeof(RawDataType)
448  / 1024.0 / 1024.0)
449  << " MB" << std::endl;
450  bool foundTheStart = false;
451  for (int idx = 0; idx < (int)stats.recentBinnedDurations.size(); ++idx)
452  {
453  if (stats.recentBinnedDurations[idx] > 0.0)
454  {
455  foundTheStart = true;
456  }
457  if (foundTheStart)
458  {
459  outStream << " " << std::fixed << std::setprecision(3)
460  << stats.recentBinnedEndTimes[idx]
461  << ": " << stats.recentBinnedSampleCounts[idx]
462  << " events at "
463  << (stats.recentBinnedSampleCounts[idx] /
464  stats.recentBinnedDurations[idx])
465  << " events/sec, data rate = "
466  << (stats.recentBinnedValueSums[idx] *
467  sizeof(RawDataType) / 1024.0 / 1024.0 /
468  stats.recentBinnedDurations[idx])
469  << " MB/sec, bin size = "
470  << stats.recentBinnedDurations[idx]
471  << " sec" << std::endl;
472  }
473  }
474  outStream.close();
475  }
476 
477  mqPtr = StatisticsCollection::getInstance().
478  getMonitoredQuantity(INCOMPLETE_EVENT_STAT_KEY);
479  if (mqPtr.get() != 0)
480  {
481  ostringstream oss;
482  oss << INCOMPLETE_EVENT_STAT_KEY << "_" << setfill('0')
483  << setw(4) << run_id_
484  << "_" << setfill('0') << setw(4) << my_rank << ".txt";
485  std::string filename = oss.str();
486  ofstream outStream(filename.c_str());
487  mqPtr->waitUntilAccumulatorsHaveBeenFlushed(3.0);
488  artdaq::MonitoredQuantityStats stats;
489  mqPtr->getStats(stats);
490  outStream << "EventStore rank " << my_rank << ": fragments processed = "
491  << stats.fullSampleCount << " at " << stats.fullSampleRate
492  << " fragments/sec, average incomplete event count = "
493  << stats.fullValueAverage << " duration = "
494  << stats.fullDuration << " sec" << std::endl
495  << " minimum incomplete event count = "
496  << stats.fullValueMin << ", maximum incomplete event count = "
497  << stats.fullValueMax << std::endl;
498  bool foundTheStart = false;
499  for (int idx = 0; idx < (int)stats.recentBinnedDurations.size(); ++idx)
500  {
501  if (stats.recentBinnedDurations[idx] > 0.0)
502  {
503  foundTheStart = true;
504  }
505  if (foundTheStart && stats.recentBinnedSampleCounts[idx] > 0.0)
506  {
507  outStream << " " << std::fixed << std::setprecision(3)
508  << stats.recentBinnedEndTimes[idx]
509  << ": " << stats.recentBinnedSampleCounts[idx]
510  << " fragments at "
511  << (stats.recentBinnedSampleCounts[idx] /
512  stats.recentBinnedDurations[idx])
513  << " fragments/sec, average incomplete event count = "
514  << (stats.recentBinnedValueSums[idx] /
515  stats.recentBinnedSampleCounts[idx])
516  << ", bin size = "
517  << stats.recentBinnedDurations[idx]
518  << " sec" << std::endl;
519  }
520  }
521  outStream << "Incomplete count now = " << events_.size() << std::endl;
522  outStream.close();
523  }
524  }
525 
526  void
527  EventStore::setup_requests_(std::string request_address)
528  {
529  if (send_requests_)
530  {
531  request_socket_ = socket(AF_INET, SOCK_DGRAM, IPPROTO_UDP);
532  if (!request_socket_)
533  {
534  TLOG_ERROR("EventStore") << "I failed to create the socket for sending Data Requests!" << TLOG_ENDL;
535  exit(1);
536  }
537  int sts = ResolveHost(request_address.c_str(), request_port_, request_addr_);
538  if (sts == -1)
539  {
540  TLOG_ERROR("EventStore") << "Unable to resolve Data Request address" << TLOG_ENDL;
541  exit(1);
542  }
543 
544  if (multicast_out_addr_ != "localhost") {
545  struct in_addr addr;
546  int sts = ResolveHost(multicast_out_addr_.c_str(), addr);
547  if (sts == -1)
548  {
549  TLOG_ERROR("EventStore") << "Unable to resolve multicast interface address" << TLOG_ENDL;
550  exit(1);
551  }
552 
553  int yes = 1;
554  if (setsockopt(request_socket_, SOL_SOCKET, SO_REUSEADDR, &yes, sizeof(yes)) < 0)
555  {
556  TLOG_ERROR("EventStore") << "Unable to enable port reuse on request socket" << TLOG_ENDL;
557  exit(1);
558  }
559  if (setsockopt(request_socket_, IPPROTO_IP, IP_MULTICAST_IF, &addr, sizeof(addr)) == -1)
560  {
561  TLOG_ERROR("EventStore") << "Cannot set outgoing interface." << TLOG_ENDL;
562  exit(1);
563  }
564  }
565  int yes = 1;
566  if (setsockopt(request_socket_, SOL_SOCKET, SO_BROADCAST, (void*)&yes, sizeof(int)) == -1)
567  {
568  TLOG_ERROR("EventStore") << "Cannot set request socket to broadcast." << TLOG_ENDL;
569  exit(1);
570  }
571  }
572  }
573 
574  void
575  EventStore::setup_tokens_()
576  {
577  if (send_routing_tokens_)
578  {
579  TLOG_DEBUG("EventStore") << "Creating Routing Token sending socket" << TLOG_ENDL;
580  token_socket_ = TCPConnect(token_address_.c_str(), token_port_);
581  if (!token_socket_)
582  {
583  TLOG_ERROR("EventStore") << "I failed to create the socket for sending Routing Tokens!" << TLOG_ENDL;
584  exit(1);
585  }
586  }
587  }
588 
589  void EventStore::do_send_request_()
590  {
591  std::this_thread::sleep_for(std::chrono::microseconds(request_delay_));
592 
593  detail::RequestMessage message;
594  {
595  std::lock_guard<std::mutex> lk(request_mutex_);
596  for (auto& req : active_requests_)
597  {
598  message.addRequest(req.first, req.second);
599  }
600  }
601  message.header()->mode = request_mode_;
602  char str[INET_ADDRSTRLEN];
603  inet_ntop(AF_INET, &(request_addr_.sin_addr), str, INET_ADDRSTRLEN);
604  TLOG_DEBUG("EventStore") << "Sending request for " << std::to_string(message.size()) << " events to multicast group " << str << TLOG_ENDL;
605  if (sendto(request_socket_, message.header(), sizeof(detail::RequestHeader), 0, (struct sockaddr *)&request_addr_, sizeof(request_addr_)) < 0)
606  {
607  TLOG_ERROR("EventStore") << "Error sending request message header" << TLOG_ENDL;
608  }
609  if (sendto(request_socket_, message.buffer(), sizeof(detail::RequestPacket) * message.size(), 0, (struct sockaddr *)&request_addr_, sizeof(request_addr_)) < 0)
610  {
611  TLOG_ERROR("EventStore") << "Error sending request message data" << TLOG_ENDL;
612  }
613  }
614 
615  void EventStore::send_routing_token_(int nSlots)
616  {
617  TLOG_DEBUG("EventStore") << "send_routing_token_ called, send_routing_tokens_=" << std::boolalpha << send_routing_tokens_ << TLOG_ENDL;
618  if (!send_routing_tokens_) return;
619  if (token_socket_ == -1) setup_tokens_();
620  detail::RoutingToken token;
621  token.header = TOKEN_MAGIC;
622  token.rank = my_rank;
623  token.new_slots_free = nSlots;
624 
625  TLOG_DEBUG("EventStore") << "Sending RoutingToken to " << token_address_ << ":" << token_port_ << TLOG_ENDL;
626  size_t sts = 0;
627  while (sts < sizeof(detail::RoutingToken)) {
628  auto res = send(token_socket_, reinterpret_cast<uint8_t*>(&token) + sts, sizeof(detail::RoutingToken) - sts, 0);
629  if (res == -1) {
630  usleep(1000);
631  continue;
632  }
633  sts += res;
634  }
635  TLOG_DEBUG("EventStore") << "Done sending RoutingToken to " << token_address_ << ":" << token_port_ << TLOG_ENDL;
636  }
637 
638  void
639  EventStore::send_request_()
640  {
641  std::thread request([=] { do_send_request_(); });
642  request.detach();
643  }
644 
645  void
647  {
648  if (metricMan)
649  {
650  metricMan->sendMetric("Incomplete Event Count", events_.size(),
651  "events", 1);
652  }
653  if (incomplete_event_report_interval_ms_ > 0 && events_.size())
654  {
655  if (std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::steady_clock::now() - last_incomplete_event_report_time_).count() < incomplete_event_report_interval_ms_) return;
656  last_incomplete_event_report_time_ = std::chrono::steady_clock::now();
657  std::ostringstream oss;
658  oss << "Incomplete Events (" << num_fragments_per_event_ << "): ";
659  for (auto& ev : events_)
660  {
661  oss << ev.first << " (" << ev.second->numFragments() << "), ";
662  }
663  TLOG_DEBUG("EventStore") << oss.str() << TLOG_ENDL;
664  }
665  }
666 }
void insert(FragmentPtr pfrag, bool printWarningWhenFragmentIsDropped=true)
Give ownership of the Fragment to the EventStore.
Definition: EventStore.cc:98
static const std::string EVENT_RATE_STAT_KEY
Key for the Event Rate MonitoredQuantity.
Definition: EventStore.hh:69
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
int TCPConnect(char const *host_in, int dflt_port, long flags=0, int sndbufsiz=0)
Connect to a host on a given port.
Definition: TCPConnect.cc:122
bool endOfData(int &readerReturnValue)
Indicate that the end of input has been reached to the art thread.
Definition: EventStore.cc:259
EventStoreInsertResult
This enumeration contains possible status codes of insertion attempts.
Definition: EventStore.hh:75
void startRun(run_id_t runID)
Start a Run.
Definition: EventStore.cc:317
EventStore()=delete
Default Constructor is deleted.
virtual ~EventStore()
EventStore Destructor.
Definition: EventStore.cc:85
The Fragment was successfully inserted.
void startSubrun()
Start a new Subrun, incrementing the subrun number.
Definition: EventStore.cc:356
The EventStore class collects Fragment objects, until it receives a complete event, at which point the event is handed over to the art thread.
Definition: EventStore.hh:49
static const std::string INCOMPLETE_EVENT_STAT_KEY
Key for the Incomplete Events MonitoredQuantity.
Definition: EventStore.hh:70
The EventStore is full, but the Fragment was accepted as it is for an already-open event...
void setSeqIDModulus(unsigned int seqIDModulus)
Set the parameter that will be used to determine which sequence IDs get grouped together into events...
Definition: EventStore.cc:274
void sendMetrics()
Send metrics to the MetricManager, if one has been instantiated in the application.
Definition: EventStore.cc:646
bool flushData()
Push any incomplete events onto the queue.
Definition: EventStore.cc:279
RawEvent::run_id_t run_id_t
Copy RawEvent::run_id_t into local scope.
Definition: EventStore.hh:64
The EventStore is full, and the Fragment was rejected.
The Fragment was rejected, because the RawEventQueue is full.
bool endRun()
Send an EndOfRunFragment to the art thread.
Definition: EventStore.cc:366
bool endSubrun()
Send an EndOfSubRunFragment to the art thread.
Definition: EventStore.cc:382