BDE 4.14.0 Production release
Loading...
Searching...
No Matches
bdlmt_eventscheduler.h
Go to the documentation of this file.
1/// @file bdlmt_eventscheduler.h
2///
3/// The content of this file has been pre-processed for Doxygen.
4///
5
6
7// bdlmt_eventscheduler.h -*-C++-*-
8#ifndef INCLUDED_BDLMT_EVENTSCHEDULER
9#define INCLUDED_BDLMT_EVENTSCHEDULER
10
11#include <bsls_ident.h>
12BSLS_IDENT("$Id: $")
13
14/// @defgroup bdlmt_eventscheduler bdlmt_eventscheduler
15/// @brief Provide a thread-safe recurring and one-time event scheduler.
16/// @addtogroup bdl
17/// @{
18/// @addtogroup bdlmt
19/// @{
20/// @addtogroup bdlmt_eventscheduler
21/// @{
22///
23/// <h1> Outline </h1>
24/// * <a href="#bdlmt_eventscheduler-purpose"> Purpose</a>
25/// * <a href="#bdlmt_eventscheduler-classes"> Classes </a>
26/// * <a href="#bdlmt_eventscheduler-description"> Description </a>
27/// * <a href="#bdlmt_eventscheduler-comparison-to-bdlmt-timereventscheduler"> Comparison to bdlmt::TimerEventScheduler </a>
28/// * <a href="#bdlmt_eventscheduler-thread-safety-and-raw-event-pointers"> Thread Safety and "Raw" Event Pointers </a>
29/// * <a href="#bdlmt_eventscheduler-the-dispatcher-thread-and-the-dispatcher-functor"> The Dispatcher Thread and the Dispatcher Functor </a>
30/// * <a href="#bdlmt_eventscheduler-timer-resolution-and-order-of-execution"> Timer Resolution and Order of Execution </a>
31/// * <a href="#bdlmt_eventscheduler-supported-clock-types"> Supported Clock Types </a>
32/// * <a href="#bdlmt_eventscheduler-scheduling-using-a-bsl-chrono-time_point"> Scheduling Using a bsl::chrono::time_point </a>
33/// * <a href="#bdlmt_eventscheduler-scheduling-using-a-bsls-timeinterval"> Scheduling Using a bsls::TimeInterval </a>
34/// * <a href="#bdlmt_eventscheduler-event-clock-substitution"> Event Clock Substitution </a>
35/// * <a href="#bdlmt_eventscheduler-thread-name-for-dispatcher-thread"> Thread Name for Dispatcher Thread </a>
36/// * <a href="#bdlmt_eventscheduler-usage"> Usage </a>
37/// * <a href="#bdlmt_eventscheduler-example-1-simple-clock"> Example 1: Simple Clock </a>
38/// * <a href="#bdlmt_eventscheduler-example-2-server-timeouts"> Example 2: Server Timeouts </a>
39/// * <a href="#bdlmt_eventscheduler-example-3-using-the-test-time-source"> Example 3: Using the Test Time Source </a>
40///
41/// # Purpose {#bdlmt_eventscheduler-purpose}
42/// Provide a thread-safe recurring and one-time event scheduler.
43///
44/// # Classes {#bdlmt_eventscheduler-classes}
45///
46/// - bdlmt::EventScheduler: a thread-safe event scheduler
47/// - bdlmt::EventSchedulerEventHandle: handle to a single scheduled event
48/// - bdlmt::EventSchedulerRecurringEventHandle: handle to a recurring event
49/// - bdlmt::EventSchedulerTestTimeSource: class for testing time changes
50///
51/// @see bdlmt_timereventscheduler
52///
53/// # Description {#bdlmt_eventscheduler-description}
54/// This component provides a thread-safe event scheduler.
55/// `bdlmt::EventScheduler`, that implements methods to schedule and cancel
56/// recurring and one-time events. All of the callbacks for these events are
57/// processed by a separate thread (called the dispatcher thread). By default
58/// the callbacks are also executed in the dispatcher thread, but that behavior
59/// can be altered by providing a dispatcher functor at creation time (see the
60/// section {The Dispatcher Thread and the Dispatcher Functor}).
61///
62/// Events may be referred to by `bdlmt::EventSchedulerEventHandle` and
63/// `bdlmt::EventSchedulerRecurringEventHandle` objects, which clean up after
64/// themselves when they go out of scope, or by `Event` and `RecurringEvent`
65/// pointers, which must be released using `releaseEventRaw`. Such pointers are
66/// used in the "Raw" API of this class and must be used carefully. Note that
67/// the Handle objects have an implicit conversion to the corresponding `Event`
68/// or `RecurringEvent` pointer types, effectively providing extra overloads for
69/// methods that take a `const Event*` to also take a `const EventHandle&`.
70///
71/// ## Comparison to bdlmt::TimerEventScheduler {#bdlmt_eventscheduler-comparison-to-bdlmt-timereventscheduler}
72///
73///
74/// This component was written after @ref bdlmt_timereventscheduler , which suffered
75/// from a couple of short-comings:
76/// 1. there was a maximum number of events it could manage, and
77/// 2. it was inefficient at dealing with large numbers of events
78///
79/// This component addresses both those problems -- there is no limit on the
80/// number of events it can manage, and it is more efficient at dealing with
81/// large numbers of events. The disadvantage of this component relative to
82/// @ref bdlmt_timereventscheduler is that handles referring to managed events in a
83/// `bdlmt::EventScheduler` are reference-counted and need to be released, while
84/// handles of events in a `bdlmt::TimerEventScheduler` are integral types that
85/// do not need to be released.
86///
87/// ## Thread Safety and "Raw" Event Pointers {#bdlmt_eventscheduler-thread-safety-and-raw-event-pointers}
88///
89///
90/// `bdlmt::EventScheduler` is thread-safe and thread-enabled, meaning that
91/// multiple threads may use their own instances of the class or use a shared
92/// instance without further synchronization. The thread safety and correct
93/// behavior of the component depend on the correct usage of `Event` pointers,
94/// which refer to scheduled events in the "Raw" API of this class. In
95/// particular:
96/// * Every `Event*` and `RecurringEvent*` populated by `scheduleEventRaw`
97/// and `scheduleRecurringEventRaw` must be released using `releaseEventRaw.`
98/// - Pointers are not released automatically when events are completed.
99/// - Pointers are not released automatically when events are canceled.
100/// - Events are not canceled when pointers to them are released.
101/// * Pointers must not be used after being released.
102/// * Pointers must never be shared or duplicated without using
103/// `addEventRefRaw` and `addRecurringEventRefRaw` to get additional
104/// references; *each* such added reference must be released separately.
105///
106/// `bdlmt::EventSchedulerEventHandle` and
107/// `bdlmt::EventSchedulerRecurringEventHandle` are **const thread-safe**. It
108/// is not safe for multiple threads to invoke non-`const` methods on the same
109/// `EventHandle` or `RecurringEventHandle` object concurrently.
110///
111/// ## The Dispatcher Thread and the Dispatcher Functor {#bdlmt_eventscheduler-the-dispatcher-thread-and-the-dispatcher-functor}
112///
113///
114/// The scheduler creates a single separate thread (called the *dispatcher*
115/// *thread*) to process all the callbacks. The dispatcher thread executes the
116/// callbacks by passing them to the dispatcher functor (optionally specified at
117/// creation time). The default dispatcher functor simply invokes the passed
118/// callback, effectively executing it in the dispatcher thread. Users can
119/// alter this behavior by defining their own dispatcher functor (for example in
120/// order to use a thread pool or a separate thread to run the callbacks). Note
121/// that the user-supplied functor will still be run in the dispatcher thread.
122///
123/// CAVEAT: Using a dispatcher functor such as the example above (to execute the
124/// callback in a separate thread) violates the guarantees of
125/// ``cancelEventAndWait()``. Users who specify a dispatcher functor that
126/// transfers the event to another thread for execution should not use
127/// ``cancelEventAndWait()``, and should instead ensure that the lifetime of any
128/// object bound to an event exceeds the lifetime of the mechanism used by the
129/// customized dispatcher functor.
130///
131/// ## Timer Resolution and Order of Execution {#bdlmt_eventscheduler-timer-resolution-and-order-of-execution}
132///
133///
134/// It is intended that recurring and one-time events are processed as closely
135/// as possible to their respective time values, and that they are processed in
136/// the order scheduled. However, this component **guarantees** only that events
137/// will not be executed before their scheduled time. Generally, events that
138/// are scheduled more than 1 microsecond apart will be executed in the order
139/// scheduled; but different behavior may be observed when events are submitted
140/// after (or shortly before) their scheduled time.
141///
142/// When events are executed in the dispatcher thread and take longer to
143/// complete than the time between events, the dispatcher can fall behind. In
144/// this case, events will be executed in the correct order as soon as the
145/// dispatcher thread becomes available; once the backlog is worked off, events
146/// will be executed at or near their scheduled times.
147///
148/// ## Supported Clock Types {#bdlmt_eventscheduler-supported-clock-types}
149///
150///
151/// An `EventScheduler` optionally accepts a clock type at construction
152/// indicating the clock by which it will internally schedule events. The clock
153/// type may be indicated by either a `bsls::SystemClockType::Enum` value, a
154/// `bsl::chrono::system_clock` object (which is equivalent to specifying
155/// `e_REALTIME`), or a `bsl::chrono::steady_clock` object (equivalent to
156/// specifying `e_MONOTONIC`). If a clock type is not specified, `e_REALTIME`
157/// is used.
158///
159/// ### Scheduling Using a bsl::chrono::time_point {#bdlmt_eventscheduler-scheduling-using-a-bsl-chrono-time_point}
160///
161///
162/// When creating either a one-time or recurring event, clients may pass a
163/// `bsl::chrono::time_point` indicating the time the event should occur. This
164/// @ref time_point object can be associated with an arbitrary clock. If the
165/// @ref time_point is associated with a different clock than was indicated at
166/// construction of the event scheduler, those time points are converted to be
167/// relative to the event scheduler's clock for processing. A possible
168/// implementation of such a conversion would be:
169/// @code
170/// bsls::TimeInterval(time - t_CLOCK::now()) + eventScheduler.now()
171/// @endcode
172/// where `time` is a @ref time_point , `t_CLOCK` is the clock associated with
173/// `time`, and `eventScheduler` is the `EventScheduler` on which the event is
174/// being scheduled. Notice that the conversion adds some imprecision and
175/// overhead to evaluation of the event. An event scheduler guarantees an event
176/// will occur at or after the supplied @ref time_point , even if that @ref time_point
177/// is defined in terms of a `t_CLOCK` different from the one used by the event
178/// scheduler. If there is a discontinuity between the clock for a @ref time_point
179/// and the event scheduler's clock, additional processing overhead may result
180/// (because the event may need to be rescheduled), and the event may also occur
181/// later than what one might otherwise expect.
182///
183/// ### Scheduling Using a bsls::TimeInterval {#bdlmt_eventscheduler-scheduling-using-a-bsls-timeinterval}
184///
185///
186/// When creating either a one-time or recurring event, clients may pass a
187/// `bsls::TimeInterval` indicating the time the event should occur as an offset
188/// from an epoch. If the clock type indicated at construction is
189/// `bsls::SystemClockType::e_REALTIME`, time should be expressed as an absolute
190/// offset since 00:00:00 UTC, January 1, 1970 (which matches the epoch used in
191/// `bdlt::CurrentTime::now(bsls::SystemClockType::e_REALTIME)`, and
192/// `bsl::chrono::system_clock::now()`). If the clock type indicated at
193/// construction is `bsls::SystemClockType::e_MONOTONIC`, time should be
194/// expressed as an absolute offset since the epoch of this clock (which matches
195/// the epoch used in
196/// `bdlt::CurrentTime::now(bsls::SystemClockType::e_MONOTONIC)` and
197/// `bsl::chrono::steady_clock`).
198///
199/// ## Event Clock Substitution {#bdlmt_eventscheduler-event-clock-substitution}
200///
201///
202/// For testing purposes, a class `bdlmt::EventSchedulerTestTimeSource` is
203/// provided to allow manual manipulation of the system-time observed by a
204/// `bdlmt::EventScheduler`. A test driver that interacts with a
205/// `bdlmt::EventScheduler` can use a `bdlmt::EventSchedulerTestTimeSource`
206/// object to control when scheduled events are triggered, allowing more
207/// reliable tests.
208///
209/// A `bdlmt::EventSchedulerTestTimeSource` can be constructed for any existing
210/// `bdlmt::EventScheduler` object that has not been started and has not had any
211/// events scheduled. When the `bdlmt::EventSchedulerTestTimeSource` is
212/// constructed, it will replace the clock of the `bdlmt::EventScheduler` to
213/// which it is attached. The internal clock of the
214/// `bdlmt::EventSchedulerTestTimeSource` will be initialized with an arbitrary
215/// value on construction, and will advance only when explicitly instructed to
216/// do so by a call to `bdlt::EventSchedulerTestTimeSource::advanceTime`. The
217/// current value of the internal clock can be accessed by calling
218/// `bdlt::EventSchedulerTestTimeSource::now`, or `bdlmt::EventScheduler::now`
219/// on the instance supplied to the `bdlmt::EventSchedulerTestTimeSource`.
220///
221/// Note that the initial value of `bdlt::EventSchedulerTestTimeSource::now` is
222/// intentionally not synchronized with `bsls::SystemTime::nowRealtimeClock`.
223/// All test events scheduled for a `bdlmt::EventScheduler` that is instrumented
224/// with a `bdlt::EventSchedulerTestTimeSource` should be scheduled in terms of
225/// an offset from whatever arbitrary time is reported by
226/// `bdlt::EventSchedulerTestTimeSource`. See
227/// @ref bdlmt_eventscheduler-example-3-using-the-test-time-source below for an illustration of how
228/// this is done.
229///
230/// ## Thread Name for Dispatcher Thread {#bdlmt_eventscheduler-thread-name-for-dispatcher-thread}
231///
232///
233/// To facilitate debugging, users can provide a thread name as the `threadName`
234/// attribute of the `bslmt::ThreadAttributes` argument passed to the `start`
235/// method, that will be used for the dispatcher thread. The thread name should
236/// not be used programmatically, but will appear in debugging tools on
237/// platforms that support naming threads to help users identify the source and
238/// purpose of a thread. If no `ThreadAttributes` object is passed, or if the
239/// `threadName` attribute is not set, the default value "bdl.EventSched" will
240/// be used.
241///
242/// ## Usage {#bdlmt_eventscheduler-usage}
243///
244///
245/// This section illustrates intended use of this component.
246///
247/// ### Example 1: Simple Clock {#bdlmt_eventscheduler-example-1-simple-clock}
248///
249///
250/// In this example we wish to log some statistics periodically. We define a
251/// method to store the value of a variable into an array, and set up a
252/// scheduler to call that as a recurring event.
253/// @code
254/// bsls::AtomicInt g_data; // Some global data we want to track
255/// typedef pair<bsls::TimeInterval, int> Value;
256///
257/// void saveData(vector<Value> *array)
258/// {
259/// array->push_back(Value(bsls::SystemTime::nowRealtimeClock(), g_data));
260/// }
261/// @endcode
262/// We allow the scheduler to run for a short time while changing this value and
263/// observe that the callback is executed:
264/// @code
265/// bdlmt::EventScheduler scheduler;
266/// vector<Value> values;
267///
268/// scheduler.scheduleRecurringEvent(bsls::TimeInterval(1.5),
269/// bdlf::BindUtil::bind(&saveData, &values)));
270/// scheduler.start();
271/// bsls::TimeInterval start = bsls::SystemTime::nowRealtimeClock();
272/// while ((bsls::SystemTime::nowRealtimeClock() -
273/// start).totalSecondsAsDouble() < 7) {
274/// ++g_data;
275/// }
276/// scheduler.stop();
277/// assert(values.size() >= 4);
278/// for (int i = 0; i < (int) values.size(); ++i) {
279/// cout << "At " << bdlt::EpochUtil::convertFromTimeInterval(
280/// values[i].first) <<
281/// " g_data was " << values[i].second << endl;
282/// }
283/// @endcode
284/// This will display, e.g.:
285/// @code
286/// At 26OCT2020_23:51:51.097283 g_data was 8008406
287/// At 26OCT2020_23:51:52.597287 g_data was 16723918
288/// At 26OCT2020_23:51:54.097269 g_data was 24563722
289/// At 26OCT2020_23:51:55.597262 g_data was 30291748
290/// @endcode
291///
292/// ### Example 2: Server Timeouts {#bdlmt_eventscheduler-example-2-server-timeouts}
293///
294///
295/// The following example shows how to use a `bdlmt::EventScheduler` to
296/// implement a timeout mechanism in a server. `my_Session` maintains several
297/// connections. It closes a connection if the data for it does not arrive
298/// before a timeout (specified at the server creation time).
299///
300/// Because the timeout is relative to the arrival of data, it is best to use a
301/// "monotonic" clock that advances at a steady rate, rather than a "wall" clock
302/// that may fluctuate to reflect real time adjustments.
303/// @code
304/// /// This class encapsulates the data and state associated with a
305/// /// connection and provides a method `processData` to process the
306/// /// incoming data for the connection.
307/// class my_Session{
308/// public:
309///
310/// /// Process the specified `data` of the specified `length`. (TBD)
311/// int processData(void *data, int length);
312/// };
313///
314/// /// This class implements a server maintaining several connections.
315/// /// A connection is closed if the data for it does not arrive
316/// /// before a timeout (specified at the server creation time).
317/// class my_Server {
318///
319/// struct Connection {
320/// bdlmt::EventSchedulerEventHandle d_timerId; // handle for timeout
321/// // event
322///
323/// my_Session *d_session_p; // session for this
324/// // connection
325/// };
326///
327/// bsl::vector<Connection*> d_connections; // maintained connections
328/// bdlmt::EventScheduler d_scheduler; // timeout event scheduler
329/// bsls::TimeInterval d_ioTimeout; // time out
330///
331/// /// Add the specified `connection` to this server and schedule
332/// /// the timeout event that closes this connection if the data
333/// /// for this connection does not arrive before the timeout.
334/// void newConnection(Connection *connection);
335///
336/// /// Close the specified `connection` and remove it from this server.
337/// void closeConnection(Connection *connection);
338///
339/// /// Return if the specified `connection` has already timed-out.
340/// /// If not, cancel the existing timeout event for the `connection`,
341/// /// process the specified `data` of the specified `length` and
342/// /// schedule a new timeout event that closes the `connection` if
343/// /// the data does not arrive before the timeout.
344/// void dataAvailable(Connection *connection, void *data, int length);
345///
346/// public:
347///
348/// /// Create a `my_Server` object with a timeout value of `ioTimeout`
349/// /// seconds. Optionally specify a `allocator` used to supply memory. If
350/// /// `allocator` is 0, the currently installed default allocator is used.
351/// my_Server(const bsls::TimeInterval& ioTimeout,
352/// bslma::Allocator *allocator = 0);
353///
354/// /// Perform the required clean-up and destroy this object.
355/// ~my_Server();
356/// };
357///
358/// my_Server::my_Server(const bsls::TimeInterval& ioTimeout,
359/// bslma::Allocator *alloc)
360/// : d_connections(alloc)
361/// , d_scheduler(bsls::SystemClockType::e_MONOTONIC, alloc)
362/// , d_ioTimeout(ioTimeout)
363/// {
364/// // TBD: logic to start monitoring the arriving connections or data
365///
366/// d_scheduler.start();
367/// }
368///
369/// my_Server::~my_Server()
370/// {
371/// // TBD: logic to clean up
372///
373/// d_scheduler.stop();
374/// }
375///
376/// void my_Server::newConnection(my_Server::Connection *connection)
377/// {
378/// // TBD: logic to add 'connection' to 'd_connections'
379///
380/// // setup the timeout for data arrival
381/// d_scheduler.scheduleEvent(
382/// &connection->d_timerId,
383/// d_scheduler.now() + d_ioTimeout,
384/// bdlf::BindUtil::bind(&my_Server::closeConnection, this, connection));
385/// }
386///
387/// void my_Server::closeConnection(my_Server::Connection *connection)
388/// {
389/// // TBD: logic to close the 'connection' and remove it from 'd_ioTimeout'
390/// }
391///
392/// void my_Server::dataAvailable(my_Server::Connection *connection,
393/// void *data,
394/// int length)
395/// {
396/// // If connection has already timed out and closed, simply return.
397/// if (d_scheduler.cancelEvent(connection->d_timerId)) {
398/// return; // RETURN
399/// }
400///
401/// // process the data
402/// connection->d_session_p->processData(data, length);
403///
404/// // setup the timeout for data arrival
405/// d_scheduler.scheduleEvent(
406/// &connection->d_timerId,
407/// d_scheduler.now() + d_ioTimeout,
408/// bdlf::BindUtil::bind(&my_Server::closeConnection, this, connection));
409/// }
410/// @endcode
411///
412/// ### Example 3: Using the Test Time Source {#bdlmt_eventscheduler-example-3-using-the-test-time-source}
413///
414///
415/// For testing purposes, the class `bdlmt::EventSchedulerTestTimeSource` is
416/// provided to allow a test to manipulate the system-time observed by a
417/// `bdlmt::EventScheduler` in order to control when events are triggered.
418/// After a scheduler is constructed, a `bdlmt::EventSchedulerTestTimeSource`
419/// object can be created atop the scheduler. A test can then use the test
420/// time-source to advance the scheduler's observed system-time in order to
421/// dispatch events in a manner coordinated by the test. Note that a
422/// `bdlmt::EventSchedulerTestTimeSource` **must** be created on an
423/// event-scheduler before any events are scheduled, or the event-scheduler is
424/// started.
425///
426/// This example shows how the clock may be altered:
427/// @code
428/// void myCallbackFunction() {
429/// puts("Event triggered!");
430/// }
431///
432/// void testCase() {
433/// // Create the scheduler
434/// bdlmt::EventScheduler scheduler;
435///
436/// // Create the time-source.
437/// // Install the time-source in the scheduler.
438/// bdlmt::EventSchedulerTestTimeSource timeSource(&scheduler);
439///
440/// // Retrieve the initial time held in the time-source.
441/// bsls::TimeInterval initialAbsoluteTime = timeSource.now();
442///
443/// // Schedule a single-run event at a 35s offset.
444/// scheduler.scheduleEvent(initialAbsoluteTime + 35,
445/// bsl::function<void()>(&myCallbackFunction));
446///
447/// // Schedule a 30s recurring event.
448/// scheduler.scheduleRecurringEvent(bsls::TimeInterval(30),
449/// bsl::function<void()>(
450/// &myCallbackFunction));
451///
452/// // Start the dispatcher thread.
453/// scheduler.start();
454///
455/// // Advance the time by 40 seconds so that each
456/// // event will run once.
457/// timeSource.advanceTime(bsls::TimeInterval(40));
458///
459/// // The line "Event triggered!" should now have
460/// // been printed to the console twice.
461///
462/// scheduler.stop();
463/// }
464/// @endcode
465/// Note that this feature should be used only for testing purposes, never in
466/// production code.
467/// @}
468/** @} */
469/** @} */
470
471/** @addtogroup bdl
472 * @{
473 */
474/** @addtogroup bdlmt
475 * @{
476 */
477/** @addtogroup bdlmt_eventscheduler
478 * @{
479 */
480
481#include <bdlscm_version.h>
482
483#include <bdlcc_skiplist.h>
484
485#include <bdlm_metricsregistry.h>
486
487#include <bdlf_bind.h>
488#include <bdlf_placeholder.h>
489
490#include <bslma_allocator.h>
492
493#include <bslmf_allocatorargt.h>
495
496#include <bslmt_condition.h>
497#include <bslmt_lockguard.h>
498#include <bslmt_mutex.h>
500#include <bslmt_threadutil.h>
501
502#include <bsls_assert.h>
503#include <bsls_atomic.h>
504#include <bsls_libraryfeatures.h>
505#include <bsls_review.h>
506#include <bsls_systemclocktype.h>
507#include <bsls_timeinterval.h>
508#include <bsls_types.h>
509
510#include <bsl_functional.h>
511#include <bsl_memory.h>
512#include <bsl_string.h>
513#include <bsl_utility.h>
514
515#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
516#include <bslmt_chronoutil.h>
517
518#include <bsl_chrono.h>
519#endif
520
521
522namespace bdlmt {
523
526class EventSchedulerTestTimeSource_Data;
527
528 // ====================
529 // class EventScheduler
530 // ====================
531
532/// This class provides a thread-safe event scheduler that executes
533/// callbacks in a separate "dispatcher thread." `start` must be invoked to
534/// start dispatching the callbacks. `stop` pauses the dispatching of the
535/// callbacks without removing the pending events.
536///
537/// See @ref bdlmt_eventscheduler
539
540 private:
541 // PRIVATE TYPES
542
543 // ================
544 // struct EventData
545 // ================
546
547 /// This `struct` encapsulates all of the information for a
548 /// non-recurring event.
549 struct EventData {
550
551 public:
552 // DATA
553
554 /// user-supplied callback invoked when associated event triggers
555 bsl::function<void()> d_callback;
556
557 /// a function that returns the difference, in microseconds, between
558 /// when the scheduled event is meant to occur and the current time
559 bsl::function<bsls::Types::Int64()> d_nowOffset;
560
561 private:
562 // NOT IMPLEMENTED
563 EventData& operator=(const EventData&);
564
565 public:
566 // TRAITS
568
569 // CREATORS
570
571 /// Create an `EventData` from the specified `callback` and
572 /// `nowOffset`. Optionally specify a `basicAllocator` used to
573 /// supply memory. If `basicAllocator` is 0, the currently
574 /// installed default allocator is used.
575 EventData(
576 const bsl::function<void()>& callback,
577 const bsl::function<bsls::Types::Int64()>& nowOffset,
578 bslma::Allocator *basicAllocator = 0)
579 : d_callback(bsl::allocator_arg, basicAllocator, callback)
580 , d_nowOffset(bsl::allocator_arg, basicAllocator, nowOffset)
581 {
582 }
583
584 /// Create an `EventData` object having the value of the specified
585 /// `original` object. Optionally specify a `basicAllocator` used
586 /// to supply memory. If `basicAllocator` is 0, the currently
587 /// installed default allocator is used.
588 EventData(const EventData& original,
589 bslma::Allocator *basicAllocator = 0)
590 : d_callback(bsl::allocator_arg, basicAllocator, original.d_callback)
591 , d_nowOffset(bsl::allocator_arg, basicAllocator, original.d_nowOffset)
592 {
593 }
594 };
595
596 // =========================
597 // struct RecurringEventData
598 // =========================
599
600 /// This `struct` encapsulates all of the information for a recurring
601 /// event.
602 struct RecurringEventData {
603
604 public:
605 // DATA
606
607 /// the time between calls (in microseconds)
608 bsls::TimeInterval d_interval;
609
610 /// user-supplied callback invoked when associated event triggers
611 bsl::function<void()> d_callback;
612
613 /// a function that returns the difference, in microseconds, between
614 /// when the scheduled event is meant to occur and the current time
615 bsl::function<bsls::Types::Int64(int)> d_nowOffset;
616
617 /// the index of the recurring event (starting with 0); passed to
618 /// `d_nowOffset` to determine the time of the next invocation of
619 /// `d_callback`
620 int d_eventIdx;
621
622 private:
623 // NOT IMPLEMENTED
624 RecurringEventData& operator=(const RecurringEventData&);
625
626 public:
627 // TRAITS
628 BSLMF_NESTED_TRAIT_DECLARATION(RecurringEventData,
630
631 // CREATORS
632
633 /// Create a `RecurringEventData` from the specified `interval`,
634 /// `callback`, and `nowOffset`. Optionally specify a
635 /// `basicAllocator` used to supply memory. If `basicAllocator` is
636 /// 0, the currently installed default allocator is used.
637 RecurringEventData(
638 const bsls::TimeInterval& interval,
639 const bsl::function<void()>& callback,
640 const bsl::function<bsls::Types::Int64(int)>& nowOffset,
641 bslma::Allocator *basicAllocator = 0)
642 : d_interval(interval)
643 , d_callback(bsl::allocator_arg, basicAllocator, callback)
644 , d_nowOffset(bsl::allocator_arg, basicAllocator, nowOffset)
645 , d_eventIdx(0)
646 {
647 }
648
649 /// Create a `RecurringEventData` object having the value of the
650 /// specified `original` object. Optionally specify a
651 /// `basicAllocator` used to supply memory. If `basicAllocator` is
652 /// 0, the currently installed default allocator is used.
653 RecurringEventData(const RecurringEventData& original,
654 bslma::Allocator *basicAllocator = 0)
655 : d_interval(original.d_interval)
656 , d_callback(bsl::allocator_arg, basicAllocator, original.d_callback)
657 , d_nowOffset(bsl::allocator_arg, basicAllocator, original.d_nowOffset)
658 , d_eventIdx(original.d_eventIdx)
659 {
660 }
661 };
662
664 RecurringEventData> RecurringEventQueue;
665
667
669
670 // FRIENDS
674
675 public:
676 // PUBLIC TYPES
677 struct Event {};
678
679 /// Pointers to the opaque structures `Event` and `RecurringEvent` are
680 /// populated by the "Raw" API of `EventScheduler`.
681 struct RecurringEvent {};
682
684
686
687 /// Defines a type alias for the dispatcher functor type.
688 typedef bsl::function<void(const bsl::function<void()>&)>
690
691 private:
692 // NOT IMPLEMENTED
694 EventScheduler& operator=(const EventScheduler&);
695
696 private:
697 // PRIVATE CLASS DATA
698 static const char s_defaultThreadName[16]; // Thread name to use when
699 // none is specified.
700
701 // PRIVATE DATA
702 CurrentTimeFunctor d_currentTimeFunctor; // when called, returns the
703 // current time the scheduler
704 // should use for the event
705 // timeline
706
707 EventQueue d_eventQueue; // events
708
709 RecurringEventQueue d_recurringQueue; // recurring events
710
711 Dispatcher d_dispatcherFunctor; // dispatch events
712
714 d_dispatcherThread; // dispatcher thread handle
715
716 bsls::AtomicUint64 d_dispatcherThreadId; // dispatcher thread id used to
717 // implement function
718 // 'isInDispatcherThread'
719
720 bslmt::Mutex d_dispatcherMutex; // serialize starting/stopping
721 // dispatcher thread
722
723 mutable bslmt::Mutex d_mutex; // synchronizes access to
724 // condition variables
725
726 bslmt::Condition d_queueCondition; // condition variable used to
727 // signal when the queues need
728 // to be checked again (when
729 // they become non-empty or get
730 // a new front member)
731
732 bslmt::Condition d_iterationCondition; // condition variable used to
733 // signal when the dispatcher
734 // is ready to enter next
735 // iteration (synchronizes
736 // 'wait' methods)
737
738 bool d_running; // controls the looping of the
739 // dispatcher thread
740
741 bool d_dispatcherAwaited; // A thread is waiting for the
742 // dispatcher to complete an
743 // iteration
744
746 *d_currentRecurringEvent;
747 // Raw reference to the
748 // scheduled event being
749 // executed
750 EventQueue::Pair *d_currentEvent;
751 // Raw reference to the
752 // scheduled recurring event
753 // being executed
754
755 unsigned int d_waitCount; // count of the number of waits
756 // performed in the main
757 // dispatch loop, used in
758 // 'advanceTime' to determine
759 // when to return
760
762 d_clockType; // clock type used
763
764 const bsl::string d_eventSchedulerName; // name of this scheduler
765
767 d_startLagHandle; // start lag metric handle
768
769 // PRIVATE CLASS METHODS
770
771 /// Return 0.
772 static bsls::Types::Int64 returnZero();
773
774 /// Return 0. The `int` argument is ignored.
775 static bsls::Types::Int64 returnZeroInt(int);
776
777#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
778 /// Return the number of microseconds between the current time and the
779 /// specified `absTime`. `absTime` is an *absolute* time represented as
780 /// an interval from some epoch, which is determined by the clock
781 /// associated with the time point. Note that this method is used when
782 /// the `t_CLOCK` type used to schedule the event differs from that of
783 /// the event scheduler itself. Also note that a negative value is
784 /// returned if `absTime` is in the past.
785 template <class t_CLOCK, class t_DURATION>
786 static bsls::Types::Int64 timeUntilTrigger(
787 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& absTime);
788
789 /// Return the number of microseconds between the current time and the
790 /// scheduled time of the specified `eventIndex`th recurring event,
791 /// which starts at the specified `absTime` and repeats at the specified
792 /// `interval`. `absTime` is an *absolute* time represented as an
793 /// interval from some epoch, which is determined by the clock
794 /// associated with the time point. The behavior is undefined unless
795 /// `0 <= eventIndex`. Note that this method is used when the `t_CLOCK`
796 /// type used to schedule the event differs from that of the event
797 /// scheduler itself.
798 template <class t_CLOCK,
799 class t_DURATION,
800 class t_REP_TYPE,
801 class t_PERIOD_TYPE>
802 static bsls::Types::Int64 timeUntilTriggerRecurring(
803 const bsl::chrono::time_point<t_CLOCK,
804 t_DURATION>& absTime,
805 const bsl::chrono::duration<t_REP_TYPE,
806 t_PERIOD_TYPE>& interval,
807 int eventIndex);
808#endif
809
810 // PRIVATE MANIPULATORS
811
812 /// Pick either `d_currentEvent` or `d_currentRecurringEvent` as the
813 /// next event to be executed, given that the current time is the
814 /// specified (absolute) `now` interval, and return the (absolute)
815 /// interval of the chosen event. If both `d_currentEvent` and
816 /// `d_currentRecurringEvent` are valid, release whichever one was not
817 /// chosen. If both `d_currentEvent` and `d_currentRecurringEvent` are
818 /// scheduled before `now`, choose `d_currentEvent`. The behavior is
819 /// undefined if neither `d_currentEvent` nor `d_currentRecurringEvent`
820 /// is valid. Note that the argument and return value of this method
821 /// are expressed in terms of the number of microseconds elapsed since
822 /// some epoch, which is determined by the clock indicated at
823 /// construction (see {Supported Clock Types} in the component
824 /// documentation). Also note that this method may update the value of
825 /// `now` with the current system time if necessary.
826 bsls::Types::Int64 chooseNextEvent(bsls::Types::Int64 *now);
827
828 /// While d_running is true, execute events in the event and recurring
829 /// event queues at their scheduled times. Note that this method
830 /// implements the dispatching thread.
831 void dispatchEvents();
832
833
834 /// Initialize this event scheduler using the stored attributes and the
835 /// specified `metricsRegistry` and `eventSchedulerName`. If
836 /// `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
837 /// used.
838 void initialize(bdlm::MetricsRegistry *metricsRegistry,
839 const bsl::string_view& eventSchedulerName);
840
841 /// Release `d_currentRecurringEvent` and `d_currentEvent`, if they
842 /// refer to valid events.
843 void releaseCurrentEvents();
844
845 /// Schedule the callback of the specified `eventData` to be dispatched
846 /// at the specified `epochTime` truncated to microseconds. Load into
847 /// the specified `event` pointer a handle that can be used to cancel
848 /// the event (by invoking `cancelEvent`). The `epochTime` is an
849 /// absolute time represented as an interval from some epoch, which is
850 /// determined by the clock indicated at construction (see {Supported
851 /// Clock Types} in the component documentation). Note that if
852 /// `epochTime` is in the past, the event is dispatched immediately.
853 void scheduleEvent(EventHandle *event,
854 const bsls::TimeInterval& epochTime,
855 const EventData& eventData);
856
857 /// Schedule the callback of the specified `eventData` to be dispatched
858 /// at the specified `epochTime` truncated to microseconds. The
859 /// `epochTime` is an absolute time represented as an interval from some
860 /// epoch, which is determined by the clock indicated at construction
861 /// (see {Supported Clock Types} in the component documentation). Note
862 /// that if `epochTime` is in the past, the event is dispatched
863 /// immediately.
864 void scheduleEvent(const bsls::TimeInterval& epochTime,
865 const EventData& eventData);
866
867 /// Schedule a recurring event that invokes the callback of the
868 /// specified `eventData` with the first event dispatched at the
869 /// specified `startEpochTime` truncated to microseconds. Load into the
870 /// specified `event` pointer a handle that can be used to cancel the
871 /// event (by invoking `cancelEvent`). The `startEpochTime` is an
872 /// absolute time represented as an interval from some epoch, which is
873 /// determined by the clock indicated at construction (see {Supported
874 /// Clock Types} in the component documentation). The behavior is
875 /// undefined unless the interval of `eventData` is at least one
876 /// microsecond. Note that if `startEpochTime` is in the past, the
877 /// first event is dispatched immediately, and additional
878 /// `(now() - startEpochTime) / eventData.d_interval` events will be
879 /// submitted serially.
880 void scheduleRecurringEvent(RecurringEventHandle *event,
881 const RecurringEventData& eventData,
882 const bsls::TimeInterval& startEpochTime);
883
884 /// Schedule a recurring event that invokes the callback of the
885 /// specified `eventData` with the first event dispatched at the
886 /// specified `startEpochTime` truncated to microseconds. Load into the
887 /// specified `event` pointer a handle that can be used to cancel the
888 /// event (by invoking `cancelEvent`). The `startEpochTime` is an
889 /// *absolute* time represented as an interval from some epoch, which is
890 /// determined by the clock indicated at construction (see {Supported
891 /// Clock Types} in the component documentation). The `event` pointer
892 /// must be released by invoking `releaseEventRaw` when it is no longer
893 /// needed. The behavior is undefined unless the interval of
894 /// `eventData` is at least one microsecond. Note that if
895 /// `startEpochTime` is in the past, the first event is dispatched
896 /// immediately, and additional
897 /// `(now() - startEpochTime) / eventData.d_interval` events will be
898 /// submitted serially.
899 void scheduleRecurringEventRaw(RecurringEvent **event,
900 const RecurringEventData& eventData,
901 const bsls::TimeInterval& startEpochTime);
902
903 public:
904 // TRAITS
906
907 // CREATORS
908
909 /// Create an event scheduler using the default dispatcher functor (see
910 /// {The Dispatcher Thread and the Dispatcher Functor} in the
911 /// component-level documentation) and using the system realtime clock
912 /// to indicate the epoch used for all time intervals. Optionally
913 /// specify a `basicAllocator` used to supply memory. If
914 /// `basicAllocator` is 0, the currently installed default allocator is
915 /// used.
916 explicit EventScheduler(bslma::Allocator *basicAllocator = 0);
917
918 /// Create an event scheduler using the default dispatcher functor (see
919 /// {The Dispatcher Thread and the Dispatcher Functor} in the
920 /// component-level documentation), using the system realtime clock to
921 /// indicate the epoch used for all time intervals, the specified
922 /// `eventSchedulerName` to be used to identify this event scheduler, and
923 /// the specified `metricsRegistry` to be used for reporting metrics.
924 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
925 /// used. Optionally specify a `basicAllocator` used to supply memory.
926 /// If `basicAllocator` is 0, the currently installed default allocator
927 /// is used.
928 explicit EventScheduler(const bsl::string_view& eventSchedulerName,
929 bdlm::MetricsRegistry *metricsRegistry,
930 bslma::Allocator *basicAllocator = 0);
931
932 /// Create an event scheduler using the default dispatcher functor (see
933 /// {The Dispatcher Thread and the Dispatcher Functor} in the
934 /// component-level documentation) and using the specified `clockType`
935 /// to indicate the epoch used for all time intervals (see {Supported
936 /// Clock Types} in the component documentation). Optionally specify a
937 /// `basicAllocator` used to supply memory. If `basicAllocator` is 0,
938 /// the currently installed default allocator is used.
940 bslma::Allocator *basicAllocator = 0);
941
942 /// Create an event scheduler using the default dispatcher functor (see
943 /// {The Dispatcher Thread and the Dispatcher Functor} in the
944 /// component-level documentation), using the specified `clockType` to
945 /// indicate the epoch used for all time intervals (see {Supported Clock
946 /// Types} in the component documentation), the specified
947 /// `eventSchedulerName` to be used to identify this event scheduler, and
948 /// the specified `metricsRegistry` to be used for reporting metrics.
949 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
950 /// used. Optionally specify a `basicAllocator` used to supply memory.
951 /// If `basicAllocator` is 0, the currently installed default allocator
952 /// is used.
954 const bsl::string_view& eventSchedulerName,
955 bdlm::MetricsRegistry *metricsRegistry,
956 bslma::Allocator *basicAllocator = 0);
957
958#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
959 /// Create an event scheduler using the default dispatcher functor (see
960 /// {The Dispatcher Thread and the Dispatcher Functor} in the
961 /// component-level documentation) and using the system realtime clock
962 /// to indicate the epoch used for all time intervals. Optionally
963 /// specify a `basicAllocator` used to supply memory. If
964 /// `basicAllocator` is 0, the currently installed default allocator is
965 /// used.
967 const bsl::chrono::system_clock&,
968 bslma::Allocator *basicAllocator = 0);
969
970 /// Create an event scheduler using the default dispatcher functor (see
971 /// {The Dispatcher Thread and the Dispatcher Functor} in the
972 /// component-level documentation), using the system realtime clock to
973 /// indicate the epoch used for all time intervals, the specified
974 /// `eventSchedulerName` to be used to identify this event scheduler, and
975 /// the specified `metricsRegistry` to be used for reporting metrics.
976 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
977 /// used. Optionally specify a `basicAllocator` used to supply memory.
978 /// If `basicAllocator` is 0, the currently installed default allocator
979 /// is used.
981 const bsl::chrono::system_clock&,
982 const bsl::string_view& eventSchedulerName,
983 bdlm::MetricsRegistry *metricsRegistry,
984 bslma::Allocator *basicAllocator = 0);
985
986 /// Create an event scheduler using the default dispatcher functor (see
987 /// {The Dispatcher Thread and the Dispatcher Functor} in the
988 /// component-level documentation) and using the system monotonic clock
989 /// to indicate the epoch used for all time intervals. Optionally
990 /// specify a `basicAllocator` used to supply memory. If
991 /// `basicAllocator` is 0, the currently installed default allocator is
992 /// used.
994 const bsl::chrono::steady_clock&,
995 bslma::Allocator *basicAllocator = 0);
996
997 /// Create an event scheduler using the default dispatcher functor (see
998 /// {The Dispatcher Thread and the Dispatcher Functor} in the
999 /// component-level documentation), using the system monotonic clock to
1000 /// indicate the epoch used for all time intervals, the specified
1001 /// `eventSchedulerName` to be used to identify this event scheduler, and
1002 /// the specified `metricsRegistry` to be used for reporting metrics.
1003 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
1004 /// used. Optionally specify a `basicAllocator` used to supply memory.
1005 /// If `basicAllocator` is 0, the currently installed default allocator
1006 /// is used.
1008 const bsl::chrono::steady_clock&,
1009 const bsl::string_view& eventSchedulerName,
1010 bdlm::MetricsRegistry *metricsRegistry,
1011 bslma::Allocator *basicAllocator = 0);
1012#endif // defined(BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY)
1013
1014 /// Create an event scheduler using the specified `dispatcherFunctor`
1015 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1016 /// component-level documentation) and using the system realtime clock
1017 /// to indicate the epoch used for all time intervals. Optionally
1018 /// specify a `basicAllocator` used to supply memory. If
1019 /// `basicAllocator` is 0, the currently installed default allocator is
1020 /// used.
1021 explicit EventScheduler(const Dispatcher& dispatcherFunctor,
1022 bslma::Allocator *basicAllocator = 0);
1023
1024 /// Create an event scheduler using the specified `dispatcherFunctor`
1025 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1026 /// component-level documentation), using the system realtime clock to
1027 /// indicate the epoch used for all time intervals, the specified
1028 /// `eventSchedulerName` to be used to identify this event scheduler, and
1029 /// the specified `metricsRegistry` to be used for reporting metrics.
1030 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
1031 /// used. Optionally specify a `basicAllocator` used to supply memory.
1032 /// If `basicAllocator` is 0, the currently installed default allocator
1033 /// is used.
1034 explicit EventScheduler(const Dispatcher& dispatcherFunctor,
1035 const bsl::string_view& eventSchedulerName,
1036 bdlm::MetricsRegistry *metricsRegistry,
1037 bslma::Allocator *basicAllocator = 0);
1038
1039 /// Create an event scheduler using the specified `dispatcherFunctor`
1040 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1041 /// component-level documentation) and using the specified `clockType`
1042 /// to indicate the epoch used for all time intervals (see {Supported
1043 /// Clock Types} in the component documentation). Optionally specify a
1044 /// `basicAllocator` used to supply memory. If `basicAllocator` is 0,
1045 /// the currently installed default allocator is used.
1046 EventScheduler(const Dispatcher& dispatcherFunctor,
1048 bslma::Allocator *basicAllocator = 0);
1049
1050 /// Create an event scheduler using the specified `dispatcherFunctor`
1051 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1052 /// component-level documentation), using the specified `clockType` to
1053 /// indicate the epoch used for all time intervals (see {Supported Clock
1054 /// Types} in the component documentation), the specified
1055 /// `eventSchedulerName` to be used to identify this event scheduler, and
1056 /// the specified `metricsRegistry` to be used for reporting metrics.
1057 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
1058 /// used. Optionally specify a `basicAllocator` used to supply memory.
1059 /// If `basicAllocator` is 0, the currently installed default allocator
1060 /// is used.
1061 EventScheduler(const Dispatcher& dispatcherFunctor,
1063 const bsl::string_view& eventSchedulerName,
1064 bdlm::MetricsRegistry *metricsRegistry,
1065 bslma::Allocator *basicAllocator = 0);
1066
1067#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1068 /// Create an event scheduler using the specified `dispatcherFunctor`
1069 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1070 /// component-level documentation) and using the system realtime clock
1071 /// to indicate the epoch used for all time intervals. Optionally
1072 /// specify a `basicAllocator` used to supply memory. If
1073 /// `basicAllocator` is 0, the currently installed default allocator is
1074 /// used.
1075 EventScheduler(const Dispatcher& dispatcherFunctor,
1076 const bsl::chrono::system_clock&,
1077 bslma::Allocator *basicAllocator = 0);
1078
1079 /// Create an event scheduler using the specified `dispatcherFunctor`
1080 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1081 /// component-level documentation), using the system realtime clock to
1082 /// indicate the epoch used for all time intervals, the specified
1083 /// `eventSchedulerName` to be used to identify this event scheduler, and
1084 /// the specified `metricsRegistry` to be used for reporting metrics.
1085 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
1086 /// used. Optionally specify a `basicAllocator` used to supply memory.
1087 /// If `basicAllocator` is 0, the currently installed default allocator
1088 /// is used.
1089 EventScheduler(const Dispatcher& dispatcherFunctor,
1090 const bsl::chrono::system_clock&,
1091 const bsl::string_view& eventSchedulerName,
1092 bdlm::MetricsRegistry *metricsRegistry,
1093 bslma::Allocator *basicAllocator = 0);
1094
1095 /// Create an event scheduler using the specified `dispatcherFunctor`
1096 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1097 /// component-level documentation) and using the system monotonic clock
1098 /// to indicate the epoch used for all time intervals. Optionally
1099 /// specify a `basicAllocator` used to supply memory. If
1100 /// `basicAllocator` is 0, the currently installed default allocator is
1101 /// used.
1102 EventScheduler(const Dispatcher& dispatcherFunctor,
1103 const bsl::chrono::steady_clock&,
1104 bslma::Allocator *basicAllocator = 0);
1105
1106 /// Create an event scheduler using the specified `dispatcherFunctor`
1107 /// (see {The Dispatcher Thread and the Dispatcher Functor} in the
1108 /// component-level documentation), using the system monotonic clock to
1109 /// indicate the epoch used for all time intervals, the specified
1110 /// `eventSchedulerName` to be used to identify this event scheduler, and
1111 /// the specified `metricsRegistry` to be used for reporting metrics.
1112 /// If `metricsRegistry` is 0, `bdlm::MetricsRegistry::singleton()` is
1113 /// used. Optionally specify a `basicAllocator` used to supply memory.
1114 /// If `basicAllocator` is 0, the currently installed default allocator
1115 /// is used.
1116 EventScheduler(const Dispatcher& dispatcherFunctor,
1117 const bsl::chrono::steady_clock&,
1118 const bsl::string_view& eventSchedulerName,
1119 bdlm::MetricsRegistry *metricsRegistry,
1120 bslma::Allocator *basicAllocator = 0);
1121#endif // defined(BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY)
1122
1123 /// Discard all unprocessed events and destroy this object. The
1124 /// behavior is undefined unless the scheduler is stopped.
1126
1127 // MANIPULATORS
1128
1129 /// Cancel all recurring and one-time events scheduled in this
1130 /// EventScheduler.
1132
1133 /// Cancel all recurring and one-time events scheduled in this
1134 /// EventScheduler. Block until all events have either been cancelled
1135 /// or dispatched before this call returns. The behavior is undefined
1136 /// if this method is invoked from the dispatcher thread.
1138
1139 int cancelEvent(const Event *handle);
1140 /// Cancel the event having the specified `handle`. Return 0 on
1141 /// successful cancellation, and a non-zero value if the `handle` is
1142 /// invalid *or* if the event has already been dispatched or canceled.
1143 /// Note that due to the implicit conversion from Handle types, these
1144 /// methods also match the following:
1145 /// @code
1146 /// int cancelEvent(const EventHandle& handle);
1147 /// int cancelEvent(const RecurringEventHandle& handle);
1148 /// @endcode
1149 /// Compared to the version taking a pointer to Handle, the managed
1150 /// reference to the event is not released until the Handle goes out of
1151 /// scope.
1152 int cancelEvent(const RecurringEvent *handle);
1153
1155 /// Cancel the event having the specified `handle` and release the
1156 /// handle. Return 0 on successful cancellation, and a non-zero value
1157 /// if the `handle` is invalid *or* if the event has already been
1158 /// dispatched or canceled. Note that `handle` is released whether this
1159 /// call is successful or not.
1161
1162 int cancelEventAndWait(const Event *handle);
1163 /// Cancel the event having the specified `handle`. Block until the
1164 /// event having `handle` (if it is valid) is either successfully
1165 /// canceled or dispatched before the call returns. Return 0 on
1166 /// successful cancellation, and a non-zero value if `handle` is invalid
1167 /// *or* if the event has already been dispatched or canceled. The
1168 /// behavior is undefined if this method is invoked from the dispatcher
1169 /// thread. Note that if the event is being executed when this method
1170 /// is invoked, this method will block until it is completed and then
1171 /// return a nonzero value.
1173
1175 /// Cancel the event having the specified `handle` and release
1176 /// `*handle`. Block until the event having `handle` (if it is valid)
1177 /// is either successfully canceled or dispatched before the call
1178 /// returns. Return 0 on successful cancellation, and a non-zero value
1179 /// if `handle` is invalid *or* if the event has already been dispatched
1180 /// or canceled. The behavior is undefined if this method is invoked
1181 /// from the dispatcher thread. Note that if the event is being
1182 /// executed when this method is invoked, this method will block until
1183 /// it is completed and then return a nonzero value. Also note that it
1184 /// is guaranteed that `*handle` will be released whether this call is
1185 /// successful or not.
1187
1188 void releaseEventRaw(Event *handle);
1189 /// Release the specified `handle`. Every handle reference added by
1190 /// `scheduleEventRaw`, `addEventRefRaw`, `scheduleRecurringEventRaw`,
1191 /// or `addRecurringEventRefRaw` must be released using this method to
1192 /// avoid leaking resources. The behavior is undefined if the value of
1193 /// `handle` is used for any purpose after being released.
1194 void releaseEventRaw(RecurringEvent *handle);
1195
1196 /// Reschedule the event referred to by the specified `handle` at the
1197 /// specified `newEpochTime` truncated to microseconds. Return 0 on
1198 /// successful reschedule, and a non-zero value if the `handle` is
1199 /// invalid *or* if the event has already been dispatched. The
1200 /// `newEpochTime` is an absolute time represented as an interval from
1201 /// some epoch, which is determined by the clock indicated at
1202 /// construction (see {Supported Clock Types} in the component
1203 /// documentation).
1204 int rescheduleEvent(const Event *handle,
1205 const bsls::TimeInterval& newEpochTime);
1206
1207#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1208 /// Reschedule the event referred to by the specified `handle` at the
1209 /// specified `newEpochTime` truncated to microseconds. Return 0 on
1210 /// successful reschedule, and a non-zero value if the `handle` is
1211 /// invalid *or* if the event has already been dispatched. The
1212 /// `newEpochTime` is an absolute time represented as an interval from
1213 /// some epoch, determined by the clock associated with the time point.
1214 template <class t_CLOCK, class t_DURATION>
1215 int rescheduleEvent(
1216 const Event *handle,
1217 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& newEpochTime);
1218#endif
1219
1220 /// Reschedule the event referred to by the specified `handle` at the
1221 /// specified `newEpochTime` truncated to microseconds. Block until the
1222 /// event having `handle` (if it is valid) is either successfully
1223 /// rescheduled or dispatched before the call returns. Return 0 on
1224 /// successful reschedule, and a non-zero value if `handle` is invalid
1225 /// *or* if the event has already been dispatched. The `newEpochTime`
1226 /// is an absolute time represented as an interval from some epoch,
1227 /// which is determined by the clock indicated at construction (see
1228 /// {Supported Clock Types} in the component documentation). The
1229 /// behavior is undefined if this method is invoked from the dispatcher
1230 /// thread.
1232 const bsls::TimeInterval& newEpochTime);
1233
1234#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1235 /// Reschedule the event referred to by the specified `handle` at the
1236 /// specified `newEpochTime` truncated to microseconds. Block until the
1237 /// event having `handle` (if it is valid) is either successfully
1238 /// rescheduled or dispatched before the call returns. Return 0 on
1239 /// successful reschedule, and a non-zero value if `handle` is invalid
1240 /// *or* if the event has already been dispatched. The `newEpochTime`
1241 /// is an absolute time represented as an interval from some epoch,
1242 /// which is determined by the clock associated with the time point.
1243 /// The behavior is undefined if this method is invoked from the
1244 /// dispatcher thread.
1245 template <class t_CLOCK, class t_DURATION>
1247 const Event *handle,
1248 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& newEpochTime);
1249#endif
1250
1251 void scheduleEvent(const bsls::TimeInterval& epochTime,
1252 const bsl::function<void()>& callback);
1253 /// Schedule the specified `callback` to be dispatched at the specified
1254 /// `epochTime` truncated to microseconds. Load into the optionally
1255 /// specified `event` a handle that can be used to cancel the event (by
1256 /// invoking `cancelEvent`). The `epochTime` is an absolute time
1257 /// represented as an interval from some epoch, which is determined by
1258 /// the clock indicated at construction (see {Supported Clock Types} in
1259 /// the component documentation). This method guarantees that the event
1260 /// will occur at or after `epochTime`. `epochTime` may be in the past,
1261 /// in which case the event will be executed as soon as possible.
1262 void scheduleEvent(EventHandle *event,
1263 const bsls::TimeInterval& epochTime,
1264 const bsl::function<void()>& callback);
1265
1266#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1267 /// Schedule the specified `callback` to be dispatched at the specified
1268 /// `epochTime` truncated to microseconds. Load into the optionally
1269 /// specified `event` a handle that can be used to cancel the event (by
1270 /// invoking `cancelEvent`). The `epochTime` is an absolute time
1271 /// represented as an interval from some epoch, which is determined by
1272 /// the clock associated with the time point. This method guarantees
1273 /// that the event will occur at or after `epochTime`. `epochTime` may
1274 /// be in the past, in which case the event will be executed as soon as
1275 /// possible.
1276 template <class t_CLOCK, class t_DURATION>
1277 void scheduleEvent(
1278 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& epochTime,
1279 const bsl::function<void()>& callback);
1280 template <class t_CLOCK, class t_DURATION>
1281 void scheduleEvent(
1282 EventHandle *event,
1283 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& epochTime,
1284 const bsl::function<void()>& callback);
1285#endif
1286
1287 /// Schedule the specified `callback` to be dispatched at the specified
1288 /// `epochTime` truncated to microseconds. Load into the specified
1289 /// `event` pointer a handle that can be used to cancel the event (by
1290 /// invoking `cancelEvent`). The `epochTime` is an *absolute* time
1291 /// represented as an interval from some epoch, which is determined by
1292 /// the clock indicated at construction (see {Supported Clock Types} in
1293 /// the component documentation). The `event` pointer must be released
1294 /// invoking `releaseEventRaw` when it is no longer needed.
1296 const bsls::TimeInterval& epochTime,
1297 const bsl::function<void()>& callback);
1298
1299#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1300 /// Schedule the specified `callback` to be dispatched at the specified
1301 /// `epochTime` truncated to microseconds. Load into the specified
1302 /// `event` pointer a handle that can be used to cancel the event (by
1303 /// invoking `cancelEvent`). The `epochTime` is an absolute time
1304 /// represented as an interval from some epoch, which is determined by
1305 /// the clock associated with the time point. The `event` pointer must
1306 /// be released invoking `releaseEventRaw` when it is no longer needed.
1307 template <class t_CLOCK, class t_DURATION>
1308 void scheduleEventRaw(
1309 Event **event,
1310 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& epochTime,
1311 const bsl::function<void()>& callback);
1312#endif
1313
1314 void scheduleRecurringEvent(const bsls::TimeInterval& interval,
1315 const bsl::function<void()>& callback,
1316 const bsls::TimeInterval& startEpochTime
1317 = bsls::TimeInterval(0));
1318 /// Schedule a recurring event that invokes the specified `callback` at
1319 /// every specified `interval` truncated to microseconds, with the first
1320 /// event dispatched at the optionally specified `startEpochTime`
1321 /// truncated to microseconds. If `startEpochTime` is not specified,
1322 /// the first event is dispatched at one `interval` from now. Load into
1323 /// the optionally specified `event` a handle that can be used to cancel
1324 /// the event (by invoking `cancelEvent`). The `startEpochTime` is an
1325 /// absolute time represented as an interval from some epoch, which is
1326 /// determined by the clock indicated at construction (see {Supported
1327 /// Clock Types} in the component documentation). The behavior is
1328 /// undefined unless `interval` is at least one microsecond. Note that
1329 /// if `startEpochTime` is in the past, the first event is dispatched
1330 /// immediately, and additional `(now() - startEpochTime) / interval`
1331 /// events will be submitted serially.
1332 void scheduleRecurringEvent(RecurringEventHandle *event,
1333 const bsls::TimeInterval& interval,
1334 const bsl::function<void()>& callback,
1335 const bsls::TimeInterval& startEpochTime
1336 = bsls::TimeInterval(0));
1337
1338#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1339 /// Schedule a recurring event that invokes the specified `callback` at
1340 /// every specified `interval` truncated to microseconds, with the first
1341 /// event dispatched at the optionally specified `startEpochTime`
1342 /// truncated to microseconds. If `startEpochTime` is not specified,
1343 /// the first event is dispatched at one `interval` from now. Load into
1344 /// the optionally specified `event` a handle that can be used to cancel
1345 /// the event (by invoking `cancelEvent`). The `startEpochTime` is an
1346 /// absolute time represented as an interval from some epoch, which is
1347 /// determined by the clock associated with the time point. The
1348 /// behavior is undefined unless `interval` is at least one microsecond.
1349 /// Note that if `startEpochTime` is in the past, the first event is
1350 /// dispatched immediately, and additional
1351 /// `(now() - startEpochTime) / interval` events will be submitted
1352 /// serially.
1353 template <class t_CLOCK,
1354 class t_REP_TYPE,
1355 class t_PERIOD_TYPE,
1356 class t_DURATION>
1357 void scheduleRecurringEvent(
1358 const bsl::chrono::duration<t_REP_TYPE,
1359 t_PERIOD_TYPE>& interval,
1360 const bsl::function<void()>& callback,
1361 const bsl::chrono::time_point<t_CLOCK,
1362 t_DURATION>& startEpochTime =
1363 t_CLOCK::now());
1364 template <class t_CLOCK,
1365 class t_REP_TYPE,
1366 class t_PERIOD_TYPE,
1367 class t_DURATION>
1368 void scheduleRecurringEvent(
1369 RecurringEventHandle *event,
1370 const bsl::chrono::duration<t_REP_TYPE,
1371 t_PERIOD_TYPE>& interval,
1372 const bsl::function<void()>& callback,
1373 const bsl::chrono::time_point<t_CLOCK,
1374 t_DURATION>& startEpochTime =
1375 t_CLOCK::now());
1376#endif
1377
1378 /// Schedule a recurring event that invokes the specified `callback` at
1379 /// every specified `interval` truncated to microseconds, with the first
1380 /// event dispatched at the optionally specified `startEpochTime`
1381 /// truncated to microseconds. If `startEpochTime` is not specified,
1382 /// the first event is dispatched at one `interval` from now. Load into
1383 /// the specified `event` pointer a handle that can be used to cancel
1384 /// the event (by invoking `cancelEvent`). The `startEpochTime` is an
1385 /// absolute time represented as an interval from some epoch, which is
1386 /// determined by the clock indicated at construction (see {Supported
1387 /// Clock Types} in the component documentation). The `event` pointer
1388 /// must be released by invoking `releaseEventRaw` when it is no longer
1389 /// needed. The behavior is undefined unless `interval` is at least one
1390 /// microsecond. Note that if `startEpochTime` is in the past, the
1391 /// first event is dispatched immediately, and additional
1392 /// `(now() - startEpochTime) / interval` events will be submitted
1393 /// serially.
1395 RecurringEvent **event,
1396 const bsls::TimeInterval& interval,
1397 const bsl::function<void()>& callback,
1398 const bsls::TimeInterval& startEpochTime
1399 = bsls::TimeInterval(0));
1400
1401#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1402 /// Schedule a recurring event that invokes the specified `callback` at
1403 /// every specified `interval` truncated to microseconds, with the first
1404 /// event dispatched at the optionally specified `startEpochTime`
1405 /// truncated to microseconds. If `startEpochTime` is not specified,
1406 /// the first event is dispatched at one `interval` from now. Load into
1407 /// the specified `event` pointer a handle that can be used to cancel
1408 /// the event (by invoking `cancelEvent`). The `startEpochTime` is an
1409 /// absolute time represented as an interval from some epoch, which is
1410 /// determined by the clock associated with the time point. The `event`
1411 /// pointer must be released by invoking `releaseEventRaw` when it is no
1412 /// longer needed. The behavior is undefined unless `interval` is at
1413 /// least one microsecond. Note that if `startEpochTime` is in the
1414 /// past, the first event is dispatched immediately, and additional
1415 /// `(now() - startEpochTime) / interval` events will be submitted
1416 /// serially.
1417 template <class t_CLOCK,
1418 class t_REP_TYPE,
1419 class t_PERIOD_TYPE,
1420 class t_DURATION>
1421 void scheduleRecurringEventRaw(
1422 RecurringEvent **event,
1423 const bsl::chrono::duration<t_REP_TYPE,
1424 t_PERIOD_TYPE>& interval,
1425 const bsl::function<void()>& callback,
1426 const bsl::chrono::time_point<t_CLOCK,
1427 t_DURATION>& startEpochTime =
1428 t_CLOCK::now());
1429#endif
1430
1431 /// Begin dispatching events on this scheduler using default attributes
1432 /// for the dispatcher thread. Return 0 on success, and a nonzero value
1433 /// otherwise. If another thread is currently executing `stop`, wait
1434 /// until the dispatcher thread stops before starting a new one. If
1435 /// this scheduler has already started (and is not currently being
1436 /// stopped by another thread) then this invocation has no effect and 0
1437 /// is returned. The created thread will use the `eventSchedulerName`
1438 /// supplied at construction if it is not empty, otherwise
1439 /// "bdl.EventSched". The behavior is undefined if this method is invoked
1440 /// in the dispatcher thread (i.e., in a job executed by this scheduler).
1441 /// Note that any event whose time has already passed is pending and
1442 /// will be dispatched immediately.
1443 int start();
1444
1445 /// Begin dispatching events on this scheduler using the specified
1446 /// `threadAttributes` for the dispatcher thread (except that the
1447 /// DETACHED attribute is ignored). Return 0 on success, and a nonzero
1448 /// value otherwise. If another thread is currently executing `stop`,
1449 /// wait until the dispatcher thread stops before starting a new one.
1450 /// If this scheduler has already started (and is not currently being
1451 /// stopped by another thread) then this invocation has no effect and 0
1452 /// is returned. The created thread will use the name
1453 /// `threadAttributes.getThreadName()` if it is not empty, otherwise
1454 /// `eventSchedulerName` supplied at construction if it is not empty,
1455 /// otherwise "bdl.EventSched". The behavior is undefined if this method
1456 /// is invoked in the dispatcher thread (i.e., in a job executed by this
1457 /// scheduler). Note that any event whose time has already passed is
1458 /// pending and will be dispatched immediately.
1459 int start(const bslmt::ThreadAttributes& threadAttributes);
1460
1461 /// End the dispatching of events on this scheduler (but do not remove
1462 /// any pending events), and wait for any (one) currently executing
1463 /// event to complete. If the scheduler is already stopped then this
1464 /// method has no effect. This scheduler can be restarted by invoking
1465 /// `start`. The behavior is undefined if this method is invoked from
1466 /// the dispatcher thread.
1467 void stop();
1468
1469 // ACCESSORS
1470
1471 /// Increment the reference count for the event referred to by the
1472 /// specified `handle` and return `handle`. There must be a
1473 /// corresponding call to `releaseEventRaw` when the reference is no
1474 /// longer needed.
1475 Event *addEventRefRaw(Event *handle) const;
1476
1477 /// Increment the reference count for the recurring event referred to by
1478 /// the specified `handle` and return `handle`. There must be a
1479 /// corresponding call to `releaseEventRaw` when the reference is no
1480 /// longer needed.
1482
1483 /// Return the value of the clock type that this object was created
1484 /// with.
1486
1487 /// Return `true` if the calling thread is the dispatcher thread of this
1488 /// scheduler, and `false` otherwise.
1489 bool isInDispatcherThread() const;
1490
1491 /// Return `true` if a call to `start` has finished successfully more
1492 /// recently than any call to `stop`, and `false` otherwise.
1493 bool isStarted() const;
1494
1495 /// Return the current epoch time, an absolute time represented as an
1496 /// interval from some epoch, which is determined by the clock indicated
1497 /// at construction (see {Supported Clock Types} in the component
1498 /// documentation).
1499 bsls::TimeInterval now() const;
1500
1501 /// Return the number of pending one-time events in this scheduler.
1502 int numEvents() const;
1503
1504 /// Return the number of recurring events registered with this
1505 /// scheduler.
1506 int numRecurringEvents() const;
1507
1508 /// Return the earliest scheduled starting time of the pending events
1509 /// and recurring events registered with this scheduler. If there are
1510 /// no pending events or recurring events, return `INT64_MAX`
1511 /// microseconds.
1513
1514 // Aspects
1515
1516 /// Return the allocator used by this object to supply memory.
1517 bslma::Allocator *allocator() const;
1518};
1519
1520 // ===============================
1521 // class EventSchedulerEventHandle
1522 // ===============================
1523
1524/// Objects of this type refer to events in the `EventScheduler` API. They
1525/// are convertible to `const Event*` references and may be used in any
1526/// method that expects them.
1528{
1529
1530 // PRIVATE TYPES
1532
1533 // DATA
1534 EventQueue::PairHandle d_handle;
1535
1536 // FRIENDS
1537 friend class EventScheduler;
1538
1539 public:
1540 // PUBLIC TYPES
1542
1543 // CREATORS
1544
1545 /// Create a new handle object that does not refer to an event.
1547
1548 /// Create a new handle object referring to the same event as the
1549 /// specified `rhs` handle.
1551
1552 /// Destroy this object and release the managed reference, if any.
1554
1555 // MANIPULATORS
1556
1557 /// Release this handle's reference, if any; then make this handle refer
1558 /// to the same event as the specified `rhs` handle. Return a
1559 /// modifiable reference to this handle.
1561
1562 /// Release the reference (if any) held by this object.
1563 void release();
1564
1565 // ACCESSORS
1566
1567 /// Return a "raw" pointer to the event managed by this handle, or 0 if
1568 /// this handle does not manage a reference.
1569 operator const Event*() const;
1570};
1571
1572 // ========================================
1573 // class EventSchedulerRecurringEventHandle
1574 // ========================================
1575
1576/// Objects of this type refer to recurring events in the `EventScheduler`
1577/// API. They are convertible to `const RecurringEvent*` references and may
1578/// be used in any method which expects these.
1580{
1581
1582 // PRIVATE TYPES
1583 typedef EventScheduler::RecurringEventData RecurringEventData;
1585 RecurringEventData> RecurringEventQueue;
1586
1587 // DATA
1589
1590 // FRIENDS
1591 friend class EventScheduler;
1592
1593 public:
1594 // PUBLIC TYPES
1596
1597 // CREATORS
1598
1599 /// Create a new handle object.
1601
1602 /// Create a new handle object referring to the same recurring event as
1603 /// the specified `rhs` handle.
1605 const EventSchedulerRecurringEventHandle& original);
1606
1607 /// Destroy this object and release the managed reference, if any.
1609
1610 // MANIPULATORS
1611
1612 /// Release the reference managed by this handle, if any; then make this
1613 /// handle refer to the same recurring event as the specified `rhs`
1614 /// handle. Return a modifiable reference to this event handle.
1617
1618 /// Release the reference managed by this handle, if any.
1619 void release();
1620
1621 // ACCESSORS
1622
1623 /// Return a "raw" pointer to the recurring event managed by this
1624 /// handle, or 0 if this handle does not manage a reference.
1625 operator const RecurringEvent*() const;
1626
1627};
1628
1629 // ==================================
1630 // class EventSchedulerTestTimeSource
1631 // ==================================
1632
1633/// This class provides a means to change the clock that is used by a given
1634/// event-scheduler to determine when events should be triggered.
1635/// Constructing a `EventSchedulerTestTimeSource` alters the behavior of the
1636/// supplied event-scheduler. After a test time-source is created, the
1637/// underlying scheduler will run events according to a discrete timeline,
1638/// whose successive values are determined by calls to `advanceTime` on the
1639/// test time-source, and can be retrieved by calling `now` on that test
1640/// time-source. Note that the "system-time" held by a test time-source
1641/// *does* *not* correspond to the current system time. Test writers must
1642/// use caution when scheduling absolute-time events so that they are
1643/// scheduled relative to the test time-source's value for `now`.
1644///
1645/// See @ref bdlmt_eventscheduler
1647
1648 private:
1649 // DATA
1651 d_data_p; // shared pointer to the state whose
1652 // lifetime must be as long as
1653 // '*this' and '*d_scheduler_p'
1654
1655 EventScheduler *d_scheduler_p; // pointer to the scheduler that we
1656 // are augmenting
1657
1658 public:
1659 // CREATORS
1660
1661 /// Create a test time-source object that will control the "system-time"
1662 /// observed by the specified `scheduler`. Initialize `now` to be an
1663 /// arbitrary time value. The behavior is undefined if any methods have
1664 /// previously been called on `scheduler`.
1665 explicit
1667
1668 // MANIPULATORS
1669
1670 /// Advance this object's current-time value by the specified `amount`
1671 /// of time, notify the scheduler that the time has changed, and wait
1672 /// for the scheduler to process the events triggered by this change in
1673 /// time. Return the updated current-time value. The behavior is
1674 /// undefined unless `amount` is positive, and `now + amount` is within
1675 /// the range that can be represented with a `bsls::TimeInterval`.
1677
1678 // ACCESSORS
1679
1680 /// Return this object's current-time value. Upon construction, this
1681 /// method will return an arbitrary value. Subsequent calls to
1682 /// `advanceTime` will adjust the arbitrary value forward.
1684};
1685
1686// ============================================================================
1687// INLINE DEFINITIONS
1688// ============================================================================
1689
1690 // -------------------------------
1691 // class EventSchedulerEventHandle
1692 // -------------------------------
1693
1694// CREATORS
1695inline
1699
1700inline
1702 const EventSchedulerEventHandle& original)
1703: d_handle(original.d_handle)
1704{
1705}
1706
1707inline
1711
1712// MANIPULATORS
1713inline
1716{
1717 d_handle = rhs.d_handle;
1718 return *this;
1719}
1720
1721inline
1723{
1724 d_handle.release();
1725}
1726} // close package namespace
1727
1728// ACCESSORS
1729
1730// The scoping of "Event" below should not be necessary, but xlc (versions 8
1731// and 9) requires it
1732inline
1733bdlmt::EventSchedulerEventHandle::
1734operator const bdlmt::EventSchedulerEventHandle::Event*() const
1735{
1736 return (const Event*)((const EventQueue::Pair*)d_handle);
1737}
1738
1739namespace bdlmt {
1740
1741 // ----------------------------------------
1742 // class EventSchedulerRecurringEventHandle
1743 // ----------------------------------------
1744
1745// CREATORS
1746inline
1750
1751inline
1757
1758inline
1762
1763// MANIPULATORS
1764inline
1766{
1767 d_handle.release();
1768}
1769
1770inline
1774{
1775 d_handle = rhs.d_handle;
1776 return *this;
1777}
1778} // close package namespace
1779
1780// ACCESSORS
1781
1782// The scoping of "RecurringEvent" below should not be necessary, but xlc
1783// (versions 8 and 9) requires it
1784inline
1785bdlmt::EventSchedulerRecurringEventHandle::operator
1787{
1788 return (const RecurringEvent*)((const RecurringEventQueue::Pair*)d_handle);
1789}
1790
1791namespace bdlmt {
1792
1793 // --------------------
1794 // class EventScheduler
1795 // --------------------
1796
1797#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1798// PRIVATE CLASS METHODS
1799template <class t_CLOCK, class t_DURATION>
1800// not inline because it gets put into a bsl::function
1801bsls::Types::Int64 EventScheduler::timeUntilTrigger(
1802 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& absTime)
1803{
1804 using namespace bsl::chrono;
1805
1806 auto now = t_CLOCK::now();
1807 microseconds offset = duration_cast<microseconds>(absTime - now);
1808 // If the time to fire is less than one microsecond in the future, then
1809 // report it as 1us.
1810 return 0 == offset.count() && absTime > now
1811 ? 1
1812 : static_cast<bsls::Types::Int64>(offset.count());
1813}
1814
1815template <class t_CLOCK,
1816 class t_DURATION,
1817 class t_REP_TYPE,
1818 class t_PERIOD_TYPE>
1819// not inline because it gets put into a bsl::function
1820bsls::Types::Int64 EventScheduler::timeUntilTriggerRecurring(
1821 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& absTime,
1822 const bsl::chrono::duration<t_REP_TYPE, t_PERIOD_TYPE>& interval,
1823 int eventIndex)
1824{
1825 BSLS_ASSERT(0 <= eventIndex);
1826
1827 return timeUntilTrigger(absTime + eventIndex * interval);
1828}
1829#endif
1830
1831// MANIPULATORS
1832inline
1834{
1835 const EventQueue::Pair *itemPtr =
1836 reinterpret_cast<const EventQueue::Pair*>(
1837 reinterpret_cast<const void*>(handle));
1838
1839 return d_eventQueue.remove(itemPtr);
1840}
1841
1842inline
1844{
1845 const RecurringEventQueue::Pair *itemPtr =
1846 reinterpret_cast<const RecurringEventQueue::Pair*>(
1847 reinterpret_cast<const void*>(handle));
1848
1849 return d_recurringQueue.remove(itemPtr);
1850}
1851
1852inline
1854{
1855 d_eventQueue.releaseReferenceRaw(reinterpret_cast<EventQueue::Pair*>(
1856 reinterpret_cast<void*>(handle)));
1857}
1858
1859inline
1861{
1862 d_recurringQueue.releaseReferenceRaw(
1863 reinterpret_cast<RecurringEventQueue::Pair*>(
1864 reinterpret_cast<void*>(handle)));
1865}
1866
1867#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1868template <class t_CLOCK, class t_DURATION>
1870 const Event *handle,
1871 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& newEpochTime)
1872{
1873 BSLS_ASSERT(handle);
1874
1875 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
1876 return rescheduleEvent(handle, newEpochTime.time_since_epoch());
1877 // RETURN
1878 }
1879
1880 const EventQueue::Pair *h = reinterpret_cast<const EventQueue::Pair *>(
1881 reinterpret_cast<const void *>(handle));
1882
1883 bool isNewTop;
1884 bslmt::LockGuard<bslmt::Mutex> lock(&d_mutex);
1885 bsls::TimeInterval offsetFromNow(newEpochTime - t_CLOCK::now());
1886
1887 if (h) {
1888 h->data().d_nowOffset = bdlf::BindUtil::bind(
1889 timeUntilTrigger<t_CLOCK, t_DURATION>,
1890 newEpochTime);
1891 }
1892
1893 int ret = d_eventQueue.updateR(h,
1894 (now() + offsetFromNow).totalMicroseconds(),
1895 &isNewTop);
1896
1897 if (0 == ret && isNewTop) {
1898 d_queueCondition.signal();
1899 }
1900 return ret;
1901}
1902
1903template <class t_CLOCK, class t_DURATION>
1905 const Event *handle,
1906 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& newEpochTime)
1907{
1908 BSLS_ASSERT(handle);
1909
1910 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
1911 return rescheduleEventAndWait(handle, newEpochTime.time_since_epoch());
1912 // RETURN
1913 }
1914
1915 int ret;
1916 const EventQueue::Pair *h =
1917 reinterpret_cast<const EventQueue::Pair *>(
1918 reinterpret_cast<const void *>(handle));
1919 {
1920 bool isNewTop;
1921 bslmt::LockGuard<bslmt::Mutex> lock(&d_mutex);
1922 bsls::TimeInterval offsetFromNow(newEpochTime - t_CLOCK::now());
1923
1924 if (h) {
1925 h->data().d_nowOffset = bdlf::BindUtil::bind(
1926 timeUntilTrigger<t_CLOCK, t_DURATION>,
1927 newEpochTime);
1928 }
1929
1930 ret = d_eventQueue.updateR(h,
1931 (now() + offsetFromNow).totalMicroseconds(),
1932 &isNewTop);
1933
1934 if (0 == ret) {
1935 if (isNewTop) {
1936 d_queueCondition.signal();
1937 }
1938 if (d_currentEvent != h) {
1939 return 0; // RETURN
1940 }
1941 }
1942 }
1943
1944 // Wait until event is rescheduled or dispatched.
1945 bslmt::LockGuard<bslmt::Mutex> lock(&d_mutex);
1946 while (1) {
1947 if (d_currentEvent != h) {
1948 break;
1949 }
1950 else {
1951 d_dispatcherAwaited = true;
1952 d_iterationCondition.wait(&d_mutex);
1953 }
1954 }
1955
1956 return ret;
1957}
1958#endif
1959
1960inline
1961void EventScheduler::scheduleEvent(const bsls::TimeInterval& epochTime,
1962 const bsl::function<void()>& callback)
1963{
1964 scheduleEvent(epochTime,
1965 EventData(callback, EventScheduler::returnZero));
1966}
1967
1968inline
1969void EventScheduler::scheduleEvent(EventHandle *event,
1970 const bsls::TimeInterval& epochTime,
1971 const bsl::function<void()>& callback)
1972{
1973 scheduleEvent(event, epochTime, EventData(callback, returnZero));
1974}
1975
1976#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
1977template <class t_CLOCK, class t_DURATION>
1978inline
1979void EventScheduler::scheduleEvent(
1980 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& epochTime,
1981 const bsl::function<void()>& callback)
1982{
1983 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
1984 scheduleEvent(epochTime.time_since_epoch(), callback);
1985 }
1986 else {
1987 bsls::TimeInterval offsetFromNow(epochTime - t_CLOCK::now());
1988 scheduleEvent(now() + offsetFromNow,
1989 EventData(callback,
1991 timeUntilTrigger<t_CLOCK, t_DURATION>,
1992 epochTime)));
1993 }
1994}
1995
1996template <class t_CLOCK, class t_DURATION>
1997void EventScheduler::scheduleEvent(
1998 EventHandle *event,
1999 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& epochTime,
2000 const bsl::function<void()>& callback)
2001{
2002 BSLS_ASSERT(event);
2003
2004 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
2005 scheduleEvent(event, epochTime.time_since_epoch(), callback);
2006 }
2007 else {
2008 bsls::TimeInterval offsetFromNow(epochTime - t_CLOCK::now());
2009
2010 scheduleEvent(event,
2011 now() + offsetFromNow,
2012 EventData(callback,
2014 timeUntilTrigger<t_CLOCK, t_DURATION>,
2015 epochTime)));
2016 }
2017}
2018
2019template <class t_CLOCK, class t_DURATION>
2021 Event **event,
2022 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& epochTime,
2023 const bsl::function<void()>& callback)
2024{
2025 BSLS_ASSERT(event);
2026
2027 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
2028 scheduleEventRaw(event, epochTime.time_since_epoch(), callback);
2029 }
2030 else {
2031 using namespace bsl::chrono;
2032
2033 microseconds stime =
2034 duration_cast<microseconds>(epochTime.time_since_epoch());
2035
2036 bool newTop;
2037 d_eventQueue.addRawR(
2038 (EventQueue::Pair **)event,
2039 (bsls::Types::Int64)stime.count(),
2040 EventData(
2041 callback,
2042 bdlf::BindUtil::bind(timeUntilTrigger<t_CLOCK, t_DURATION>,
2043 epochTime)),
2044 &newTop);
2045
2046 if (newTop) {
2047 bslmt::LockGuard<bslmt::Mutex> lock(&d_mutex);
2048 d_queueCondition.signal();
2049 }
2050 }
2051}
2052#endif
2053
2054inline
2055void EventScheduler::scheduleRecurringEvent(
2056 const bsls::TimeInterval& interval,
2057 const bsl::function<void()>& callback,
2058 const bsls::TimeInterval& startEpochTime)
2059{
2060 // Note that when this review is converted to an assert, the following
2061 // assert is redundant and can be removed.
2062 BSLS_REVIEW(1 <= interval.totalMicroseconds());
2063 BSLS_ASSERT(0 != interval);
2064
2065 scheduleRecurringEventRaw(0, interval, callback, startEpochTime);
2066}
2067
2068inline
2069void EventScheduler::scheduleRecurringEvent(
2070 RecurringEventHandle *event,
2071 const bsls::TimeInterval& interval,
2072 const bsl::function<void()>& callback,
2073 const bsls::TimeInterval& startEpochTime)
2074{
2075 // Note that when this review is converted to an assert, the following
2076 // assert is redundant and can be removed.
2077 BSLS_REVIEW(1 <= interval.totalMicroseconds());
2078 BSLS_ASSERT(0 != interval);
2079 BSLS_ASSERT(event);
2080
2081 scheduleRecurringEvent(
2082 event,
2083 RecurringEventData(interval, callback, returnZeroInt),
2084 startEpochTime);
2085}
2086
2087#ifdef BSLS_LIBRARYFEATURES_HAS_CPP11_BASELINE_LIBRARY
2088template <class t_CLOCK,
2089 class t_REP_TYPE,
2090 class t_PERIOD_TYPE,
2091 class t_DURATION>
2092void EventScheduler::scheduleRecurringEvent(
2093 const bsl::chrono::duration<t_REP_TYPE, t_PERIOD_TYPE>& interval,
2094 const bsl::function<void()>& callback,
2095 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& startEpochTime)
2096{
2097 BSLS_ASSERT(bsl::chrono::microseconds(1) <= interval);
2098
2099 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
2100 scheduleRecurringEvent(interval,
2101 callback,
2102 startEpochTime.time_since_epoch());
2103 }
2104 else {
2105 using namespace bsl::chrono;
2106
2107 bsls::TimeInterval offsetFromNow(startEpochTime - t_CLOCK::now());
2108
2109 scheduleRecurringEventRaw(
2110 0,
2111 RecurringEventData(
2112 interval,
2113 callback,
2114 bdlf::BindUtil::bind(timeUntilTriggerRecurring<t_CLOCK,
2115 t_DURATION,
2116 t_REP_TYPE,
2117 t_PERIOD_TYPE>,
2118 startEpochTime,
2119 interval,
2121 now() + offsetFromNow);
2122 }
2123}
2124
2125template <class t_CLOCK,
2126 class t_REP_TYPE,
2127 class t_PERIOD_TYPE,
2128 class t_DURATION>
2129void EventScheduler::scheduleRecurringEvent(
2130 RecurringEventHandle *event,
2131 const bsl::chrono::duration<t_REP_TYPE, t_PERIOD_TYPE>& interval,
2132 const bsl::function<void()>& callback,
2133 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& startEpochTime)
2134{
2135 BSLS_ASSERT(event);
2136 BSLS_ASSERT(bsl::chrono::microseconds(1) <= interval);
2137
2138 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
2139 scheduleRecurringEvent(event,
2140 interval,
2141 callback,
2142 startEpochTime.time_since_epoch());
2143 }
2144 else {
2145 using namespace bsl::chrono;
2146
2147 bsls::TimeInterval offsetFromNow(startEpochTime - t_CLOCK::now());
2148
2149 scheduleRecurringEvent(
2150 event,
2151 RecurringEventData(
2152 interval,
2153 callback,
2154 bdlf::BindUtil::bind(timeUntilTriggerRecurring<t_CLOCK,
2155 t_DURATION,
2156 t_REP_TYPE,
2157 t_PERIOD_TYPE>,
2158 startEpochTime,
2159 interval,
2161 now() + offsetFromNow);
2162 }
2163}
2164
2165template <class t_CLOCK,
2166 class t_REP_TYPE,
2167 class t_PERIOD_TYPE,
2168 class t_DURATION>
2169void EventScheduler::scheduleRecurringEventRaw(
2170 RecurringEvent **event,
2171 const bsl::chrono::duration<t_REP_TYPE, t_PERIOD_TYPE>& interval,
2172 const bsl::function<void()>& callback,
2173 const bsl::chrono::time_point<t_CLOCK, t_DURATION>& startEpochTime)
2174{
2175 BSLS_ASSERT(event);
2176 BSLS_ASSERT(bsl::chrono::microseconds(1) <= interval);
2177
2178 if (bslmt::ChronoUtil::isMatchingClock<t_CLOCK>(d_clockType)) {
2179 scheduleRecurringEventRaw(event,
2180 interval,
2181 callback,
2182 startEpochTime.time_since_epoch());
2183 }
2184 else {
2185 using namespace bsl::chrono;
2186
2187 bsls::TimeInterval offsetFromNow(startEpochTime - t_CLOCK::now());
2188
2189 scheduleRecurringEventRaw(
2190 event,
2191 RecurringEventData(
2192 interval,
2193 callback,
2194 bdlf::BindUtil::bind(timeUntilTriggerRecurring<t_CLOCK,
2195 t_DURATION,
2196 t_REP_TYPE,
2197 t_PERIOD_TYPE>,
2198 startEpochTime,
2199 interval,
2201 now() + offsetFromNow);
2202 }
2203}
2204#endif
2205
2206// ACCESSORS
2207inline
2210{
2211 EventQueue::Pair *h = reinterpret_cast<EventQueue::Pair*>(
2212 reinterpret_cast<void*>(handle));
2213 return reinterpret_cast<Event*>(d_eventQueue.addPairReferenceRaw(h));
2214}
2215
2216inline
2219{
2221 reinterpret_cast<RecurringEventQueue::Pair*>(
2222 reinterpret_cast<void*>(handle));
2223 return reinterpret_cast<RecurringEvent*>(
2224 d_recurringQueue.addPairReferenceRaw(h));
2225}
2226
2227inline
2229{
2230 return d_clockType;
2231}
2232
2233inline
2235{
2236 return d_dispatcherThreadId.loadAcquire() ==
2238}
2239
2240inline
2242{
2243 return d_currentTimeFunctor();
2244}
2245
2246inline
2248{
2249 return d_eventQueue.length();
2250}
2251
2252inline
2254{
2255 return d_recurringQueue.length();
2256}
2257
2258 // Aspects
2259
2260inline
2262{
2263 return d_eventQueue.allocator();
2264}
2265
2266} // close package namespace
2267
2268
2269#endif
2270
2271// ----------------------------------------------------------------------------
2272// Copyright 2024 Bloomberg Finance L.P.
2273//
2274// Licensed under the Apache License, Version 2.0 (the "License");
2275// you may not use this file except in compliance with the License.
2276// You may obtain a copy of the License at
2277//
2278// http://www.apache.org/licenses/LICENSE-2.0
2279//
2280// Unless required by applicable law or agreed to in writing, software
2281// distributed under the License is distributed on an "AS IS" BASIS,
2282// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
2283// See the License for the specific language governing permissions and
2284// limitations under the License.
2285// ----------------------------- END-OF-FILE ----------------------------------
2286
2287/** @} */
2288/** @} */
2289/** @} */
#define BSLMF_NESTED_TRAIT_DECLARATION(t_TYPE, t_TRAIT)
Definition bslmf_nestedtraitdeclaration.h:231
Definition bdlcc_skiplist.h:693
void release()
Release the reference (if any) managed by this SkipListPairHandle.
Definition bdlcc_skiplist.h:1852
Definition bdlcc_skiplist.h:657
DATA & data() const
Return a reference to the modifiable "data" of this pair.
Definition bdlcc_skiplist.h:1786
Definition bdlcc_skiplist.h:783
int remove(const Pair *reference)
Definition bdlcc_skiplist.h:3235
bslma::Allocator * allocator() const
Return the allocator used by this object to supply memory.
Definition bdlcc_skiplist.h:3772
int length() const
Return the number of items in this list.
Definition bdlcc_skiplist.h:3419
void addRawR(Pair **result, const KEY &key, const DATA &data, bool *newFrontFlag=0)
Definition bdlcc_skiplist.h:3148
int updateR(const Pair *reference, const KEY &newKey, bool *newFrontFlag=0, bool allowDuplicates=true)
Definition bdlcc_skiplist.h:3324
static Bind< bslmf::Nil, t_FUNC, Bind_BoundTuple0 > bind(t_FUNC func)
Definition bdlf_bind.h:1830
Definition bdlm_metricsregistry.h:287
Definition bdlm_metricsregistry.h:199
Definition bdlmt_eventscheduler.h:1528
EventSchedulerEventHandle & operator=(const EventSchedulerEventHandle &rhs)
Definition bdlmt_eventscheduler.h:1715
EventScheduler::Event Event
Definition bdlmt_eventscheduler.h:1541
EventSchedulerEventHandle()
Create a new handle object that does not refer to an event.
Definition bdlmt_eventscheduler.h:1696
~EventSchedulerEventHandle()
Destroy this object and release the managed reference, if any.
Definition bdlmt_eventscheduler.h:1708
void release()
Release the reference (if any) held by this object.
Definition bdlmt_eventscheduler.h:1722
Definition bdlmt_eventscheduler.h:1580
EventSchedulerRecurringEventHandle()
Create a new handle object.
Definition bdlmt_eventscheduler.h:1747
void release()
Release the reference managed by this handle, if any.
Definition bdlmt_eventscheduler.h:1765
~EventSchedulerRecurringEventHandle()
Destroy this object and release the managed reference, if any.
Definition bdlmt_eventscheduler.h:1759
EventScheduler::RecurringEvent RecurringEvent
Definition bdlmt_eventscheduler.h:1595
EventSchedulerRecurringEventHandle & operator=(const EventSchedulerRecurringEventHandle &rhs)
Definition bdlmt_eventscheduler.h:1772
Definition bdlmt_eventscheduler.h:1646
EventSchedulerTestTimeSource(EventScheduler *scheduler)
bsls::TimeInterval advanceTime(bsls::TimeInterval amount)
bsls::TimeInterval now() const
Definition bdlmt_eventscheduler.h:538
EventScheduler(const Dispatcher &dispatcherFunctor, const bsl::chrono::steady_clock &, const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
EventScheduler(bsls::SystemClockType::Enum clockType, const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
bsls::TimeInterval now() const
Definition bdlmt_eventscheduler.h:2241
int cancelEvent(RecurringEventHandle *handle)
int cancelEventAndWait(const Event *handle)
int rescheduleEvent(const Event *handle, const bsls::TimeInterval &newEpochTime)
EventScheduler(const Dispatcher &dispatcherFunctor, const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
EventScheduler(const Dispatcher &dispatcherFunctor, bsls::SystemClockType::Enum clockType, bslma::Allocator *basicAllocator=0)
bool isInDispatcherThread() const
Definition bdlmt_eventscheduler.h:2234
RecurringEvent * addRecurringEventRefRaw(RecurringEvent *handle) const
Definition bdlmt_eventscheduler.h:2218
int cancelEvent(EventHandle *handle)
int numRecurringEvents() const
Definition bdlmt_eventscheduler.h:2253
bslma::Allocator * allocator() const
Return the allocator used by this object to supply memory.
Definition bdlmt_eventscheduler.h:2261
EventScheduler(const bsl::chrono::steady_clock &, const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
bsls::TimeInterval nextPendingEventTime() const
EventScheduler(bslma::Allocator *basicAllocator=0)
int start(const bslmt::ThreadAttributes &threadAttributes)
void releaseEventRaw(Event *handle)
Definition bdlmt_eventscheduler.h:1853
EventSchedulerEventHandle EventHandle
Definition bdlmt_eventscheduler.h:683
int cancelEvent(const Event *handle)
Definition bdlmt_eventscheduler.h:1833
EventScheduler(const Dispatcher &dispatcherFunctor, bslma::Allocator *basicAllocator=0)
int cancelEventAndWait(const RecurringEvent *handle)
EventScheduler(const Dispatcher &dispatcherFunctor, const bsl::chrono::system_clock &, const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
EventScheduler(const bsl::chrono::steady_clock &, bslma::Allocator *basicAllocator=0)
EventScheduler(const bsl::chrono::system_clock &, bslma::Allocator *basicAllocator=0)
int cancelEventAndWait(RecurringEventHandle *handle)
EventSchedulerRecurringEventHandle RecurringEventHandle
Definition bdlmt_eventscheduler.h:685
EventScheduler(const bsl::chrono::system_clock &, const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
bsls::SystemClockType::Enum clockType() const
Definition bdlmt_eventscheduler.h:2228
EventScheduler(const Dispatcher &dispatcherFunctor, const bsl::chrono::system_clock &, bslma::Allocator *basicAllocator=0)
BSLMF_NESTED_TRAIT_DECLARATION(EventScheduler, bslma::UsesBslmaAllocator)
int cancelEventAndWait(EventHandle *handle)
EventScheduler(const Dispatcher &dispatcherFunctor, bsls::SystemClockType::Enum clockType, const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
EventScheduler(bsls::SystemClockType::Enum clockType, bslma::Allocator *basicAllocator=0)
EventScheduler(const bsl::string_view &eventSchedulerName, bdlm::MetricsRegistry *metricsRegistry, bslma::Allocator *basicAllocator=0)
void scheduleEventRaw(Event **event, const bsls::TimeInterval &epochTime, const bsl::function< void()> &callback)
int rescheduleEventAndWait(const Event *handle, const bsls::TimeInterval &newEpochTime)
bsl::function< void(const bsl::function< void()> &)> Dispatcher
Defines a type alias for the dispatcher functor type.
Definition bdlmt_eventscheduler.h:689
void scheduleRecurringEventRaw(RecurringEvent **event, const bsls::TimeInterval &interval, const bsl::function< void()> &callback, const bsls::TimeInterval &startEpochTime=bsls::TimeInterval(0))
bool isStarted() const
EventScheduler(const Dispatcher &dispatcherFunctor, const bsl::chrono::steady_clock &, bslma::Allocator *basicAllocator=0)
Event * addEventRefRaw(Event *handle) const
Definition bdlmt_eventscheduler.h:2209
int numEvents() const
Return the number of pending one-time events in this scheduler.
Definition bdlmt_eventscheduler.h:2247
Definition bslstl_stringview.h:441
Definition bslstl_string.h:1281
Forward declaration.
Definition bslstl_function.h:934
Definition bslstl_sharedptr.h:1830
Definition bslma_allocator.h:457
Definition bslmt_condition.h:220
int wait(Mutex *mutex)
Definition bslmt_condition.h:423
void signal()
Definition bslmt_condition.h:395
Definition bslmt_lockguard.h:234
Definition bslmt_mutex.h:315
Definition bslmt_threadattributes.h:356
Definition bsls_atomic.h:1195
Types::Uint64 loadAcquire() const
Definition bsls_atomic.h:2249
Definition bsls_timeinterval.h:301
BSLS_KEYWORD_CONSTEXPR_CPP14 bsls::Types::Int64 totalMicroseconds() const
Definition bsls_timeinterval.h:1397
#define BSLS_ASSERT(X)
Definition bsls_assert.h:1804
#define BSLS_IDENT(str)
Definition bsls_ident.h:195
#define BSLS_REVIEW(X)
Definition bsls_review.h:949
const PlaceHolder< 1 > _1
Definition bdlmt_eventscheduler.h:522
Definition bslstl_chrono.h:146
Definition bdlmt_eventscheduler.h:677
Definition bdlmt_eventscheduler.h:681
Definition bslma_usesbslmaallocator.h:343
Imp::Handle Handle
Definition bslmt_threadutil.h:385
static bsls::Types::Uint64 selfIdAsUint64()
Definition bslmt_threadutil.h:1100
Enum
Definition bsls_systemclocktype.h:117
long long Int64
Definition bsls_types.h:132