tokio/runtime/time/
mod.rs

1// Currently, rust warns when an unsafe fn contains an unsafe {} block. However,
2// in the future, this will change to the reverse. For now, suppress this
3// warning and generally stick with being explicit about unsafety.
4#![allow(unused_unsafe)]
5#![cfg_attr(not(feature = "rt"), allow(dead_code))]
6
7//! Time driver.
8
9mod entry;
10pub(crate) use entry::TimerEntry;
11use entry::{EntryList, TimerHandle, TimerShared, MAX_SAFE_MILLIS_DURATION};
12
13mod handle;
14pub(crate) use self::handle::Handle;
15use self::wheel::Wheel;
16
17mod source;
18pub(crate) use source::TimeSource;
19
20mod wheel;
21
22use crate::loom::sync::atomic::{AtomicBool, Ordering};
23use crate::loom::sync::{Mutex, RwLock};
24use crate::runtime::driver::{self, IoHandle, IoStack};
25use crate::time::error::Error;
26use crate::time::{Clock, Duration};
27use crate::util::WakeList;
28
29use crate::loom::sync::atomic::AtomicU64;
30use std::fmt;
31use std::{num::NonZeroU64, ptr::NonNull};
32
33struct AtomicOptionNonZeroU64(AtomicU64);
34
35// A helper type to store the `next_wake`.
36impl AtomicOptionNonZeroU64 {
37    fn new(val: Option<NonZeroU64>) -> Self {
38        Self(AtomicU64::new(val.map_or(0, NonZeroU64::get)))
39    }
40
41    fn store(&self, val: Option<NonZeroU64>) {
42        self.0
43            .store(val.map_or(0, NonZeroU64::get), Ordering::Relaxed);
44    }
45
46    fn load(&self) -> Option<NonZeroU64> {
47        NonZeroU64::new(self.0.load(Ordering::Relaxed))
48    }
49}
50
51/// Time implementation that drives [`Sleep`][sleep], [`Interval`][interval], and [`Timeout`][timeout].
52///
53/// A `Driver` instance tracks the state necessary for managing time and
54/// notifying the [`Sleep`][sleep] instances once their deadlines are reached.
55///
56/// It is expected that a single instance manages many individual [`Sleep`][sleep]
57/// instances. The `Driver` implementation is thread-safe and, as such, is able
58/// to handle callers from across threads.
59///
60/// After creating the `Driver` instance, the caller must repeatedly call `park`
61/// or `park_timeout`. The time driver will perform no work unless `park` or
62/// `park_timeout` is called repeatedly.
63///
64/// The driver has a resolution of one millisecond. Any unit of time that falls
65/// between milliseconds are rounded up to the next millisecond.
66///
67/// When an instance is dropped, any outstanding [`Sleep`][sleep] instance that has not
68/// elapsed will be notified with an error. At this point, calling `poll` on the
69/// [`Sleep`][sleep] instance will result in panic.
70///
71/// # Implementation
72///
73/// The time driver is based on the [paper by Varghese and Lauck][paper].
74///
75/// A hashed timing wheel is a vector of slots, where each slot handles a time
76/// slice. As time progresses, the timer walks over the slot for the current
77/// instant, and processes each entry for that slot. When the timer reaches the
78/// end of the wheel, it starts again at the beginning.
79///
80/// The implementation maintains six wheels arranged in a set of levels. As the
81/// levels go up, the slots of the associated wheel represent larger intervals
82/// of time. At each level, the wheel has 64 slots. Each slot covers a range of
83/// time equal to the wheel at the lower level. At level zero, each slot
84/// represents one millisecond of time.
85///
86/// The wheels are:
87///
88/// * Level 0: 64 x 1 millisecond slots.
89/// * Level 1: 64 x 64 millisecond slots.
90/// * Level 2: 64 x ~4 second slots.
91/// * Level 3: 64 x ~4 minute slots.
92/// * Level 4: 64 x ~4 hour slots.
93/// * Level 5: 64 x ~12 day slots.
94///
95/// When the timer processes entries at level zero, it will notify all the
96/// `Sleep` instances as their deadlines have been reached. For all higher
97/// levels, all entries will be redistributed across the wheel at the next level
98/// down. Eventually, as time progresses, entries with [`Sleep`][sleep] instances will
99/// either be canceled (dropped) or their associated entries will reach level
100/// zero and be notified.
101///
102/// [paper]: http://www.cs.columbia.edu/~nahum/w6998/papers/ton97-timing-wheels.pdf
103/// [sleep]: crate::time::Sleep
104/// [timeout]: crate::time::Timeout
105/// [interval]: crate::time::Interval
106#[derive(Debug)]
107pub(crate) struct Driver {
108    /// Parker to delegate to.
109    park: IoStack,
110}
111
112/// Timer state shared between `Driver`, `Handle`, and `Registration`.
113struct Inner {
114    /// The earliest time at which we promise to wake up without unparking.
115    next_wake: AtomicOptionNonZeroU64,
116
117    /// Sharded Timer wheels.
118    wheels: RwLock<ShardedWheel>,
119
120    /// Number of entries in the sharded timer wheels.
121    wheels_len: u32,
122
123    /// True if the driver is being shutdown.
124    pub(super) is_shutdown: AtomicBool,
125
126    // When `true`, a call to `park_timeout` should immediately return and time
127    // should not advance. One reason for this to be `true` is if the task
128    // passed to `Runtime::block_on` called `task::yield_now()`.
129    //
130    // While it may look racy, it only has any effect when the clock is paused
131    // and pausing the clock is restricted to a single-threaded runtime.
132    #[cfg(feature = "test-util")]
133    did_wake: AtomicBool,
134}
135
136/// Wrapper around the sharded timer wheels.
137struct ShardedWheel(Box<[Mutex<wheel::Wheel>]>);
138
139// ===== impl Driver =====
140
141impl Driver {
142    /// Creates a new `Driver` instance that uses `park` to block the current
143    /// thread and `time_source` to get the current time and convert to ticks.
144    ///
145    /// Specifying the source of time is useful when testing.
146    pub(crate) fn new(park: IoStack, clock: &Clock, shards: u32) -> (Driver, Handle) {
147        assert!(shards > 0);
148
149        let time_source = TimeSource::new(clock);
150        let wheels: Vec<_> = (0..shards)
151            .map(|_| Mutex::new(wheel::Wheel::new()))
152            .collect();
153
154        let handle = Handle {
155            time_source,
156            inner: Inner {
157                next_wake: AtomicOptionNonZeroU64::new(None),
158                wheels: RwLock::new(ShardedWheel(wheels.into_boxed_slice())),
159                wheels_len: shards,
160                is_shutdown: AtomicBool::new(false),
161                #[cfg(feature = "test-util")]
162                did_wake: AtomicBool::new(false),
163            },
164        };
165
166        let driver = Driver { park };
167
168        (driver, handle)
169    }
170
171    pub(crate) fn park(&mut self, handle: &driver::Handle) {
172        self.park_internal(handle, None);
173    }
174
175    pub(crate) fn park_timeout(&mut self, handle: &driver::Handle, duration: Duration) {
176        self.park_internal(handle, Some(duration));
177    }
178
179    pub(crate) fn shutdown(&mut self, rt_handle: &driver::Handle) {
180        let handle = rt_handle.time();
181
182        if handle.is_shutdown() {
183            return;
184        }
185
186        handle.inner.is_shutdown.store(true, Ordering::SeqCst);
187
188        // Advance time forward to the end of time.
189
190        handle.process_at_time(0, u64::MAX);
191
192        self.park.shutdown(rt_handle);
193    }
194
195    fn park_internal(&mut self, rt_handle: &driver::Handle, limit: Option<Duration>) {
196        let handle = rt_handle.time();
197        assert!(!handle.is_shutdown());
198
199        // Finds out the min expiration time to park.
200        let expiration_time = {
201            let mut wheels_lock = rt_handle.time().inner.wheels.write();
202            let expiration_time = wheels_lock
203                .0
204                .iter_mut()
205                .filter_map(|wheel| wheel.get_mut().next_expiration_time())
206                .min();
207
208            rt_handle
209                .time()
210                .inner
211                .next_wake
212                .store(next_wake_time(expiration_time));
213
214            expiration_time
215        };
216
217        match expiration_time {
218            Some(when) => {
219                let now = handle.time_source.now(rt_handle.clock());
220                // Note that we effectively round up to 1ms here - this avoids
221                // very short-duration microsecond-resolution sleeps that the OS
222                // might treat as zero-length.
223                let mut duration = handle
224                    .time_source
225                    .tick_to_duration(when.saturating_sub(now));
226
227                if duration > Duration::from_millis(0) {
228                    if let Some(limit) = limit {
229                        duration = std::cmp::min(limit, duration);
230                    }
231
232                    self.park_thread_timeout(rt_handle, duration);
233                } else {
234                    self.park.park_timeout(rt_handle, Duration::from_secs(0));
235                }
236            }
237            None => {
238                if let Some(duration) = limit {
239                    self.park_thread_timeout(rt_handle, duration);
240                } else {
241                    self.park.park(rt_handle);
242                }
243            }
244        }
245
246        // Process pending timers after waking up
247        handle.process(rt_handle.clock());
248    }
249
250    cfg_test_util! {
251        fn park_thread_timeout(&mut self, rt_handle: &driver::Handle, duration: Duration) {
252            let handle = rt_handle.time();
253            let clock = rt_handle.clock();
254
255            if clock.can_auto_advance() {
256                self.park.park_timeout(rt_handle, Duration::from_secs(0));
257
258                // If the time driver was woken, then the park completed
259                // before the "duration" elapsed (usually caused by a
260                // yield in `Runtime::block_on`). In this case, we don't
261                // advance the clock.
262                if !handle.did_wake() {
263                    // Simulate advancing time
264                    if let Err(msg) = clock.advance(duration) {
265                        panic!("{}", msg);
266                    }
267                }
268            } else {
269                self.park.park_timeout(rt_handle, duration);
270            }
271        }
272    }
273
274    cfg_not_test_util! {
275        fn park_thread_timeout(&mut self, rt_handle: &driver::Handle, duration: Duration) {
276            self.park.park_timeout(rt_handle, duration);
277        }
278    }
279}
280
281// Helper function to turn expiration_time into next_wake_time.
282// Since the `park_timeout` will round up to 1ms for avoiding very
283// short-duration microsecond-resolution sleeps, we do the same here.
284// The conversion is as follows
285// None => None
286// Some(0) => Some(1)
287// Some(i) => Some(i)
288fn next_wake_time(expiration_time: Option<u64>) -> Option<NonZeroU64> {
289    expiration_time.and_then(|v| {
290        if v == 0 {
291            NonZeroU64::new(1)
292        } else {
293            NonZeroU64::new(v)
294        }
295    })
296}
297
298impl Handle {
299    /// Runs timer related logic, and returns the next wakeup time
300    pub(self) fn process(&self, clock: &Clock) {
301        let now = self.time_source().now(clock);
302        // For fairness, randomly select one to start.
303        let shards = self.inner.get_shard_size();
304        let start = crate::runtime::context::thread_rng_n(shards);
305        self.process_at_time(start, now);
306    }
307
308    pub(self) fn process_at_time(&self, start: u32, now: u64) {
309        let shards = self.inner.get_shard_size();
310
311        let expiration_time = (start..shards + start)
312            .filter_map(|i| self.process_at_sharded_time(i, now))
313            .min();
314
315        self.inner.next_wake.store(next_wake_time(expiration_time));
316    }
317
318    // Returns the next wakeup time of this shard.
319    pub(self) fn process_at_sharded_time(&self, id: u32, mut now: u64) -> Option<u64> {
320        let mut waker_list = WakeList::new();
321        let mut wheels_lock = self.inner.wheels.read();
322        let mut lock = wheels_lock.lock_sharded_wheel(id);
323
324        if now < lock.elapsed() {
325            // Time went backwards! This normally shouldn't happen as the Rust language
326            // guarantees that an Instant is monotonic, but can happen when running
327            // Linux in a VM on a Windows host due to std incorrectly trusting the
328            // hardware clock to be monotonic.
329            //
330            // See <https://github.com/tokio-rs/tokio/issues/3619> for more information.
331            now = lock.elapsed();
332        }
333
334        while let Some(entry) = lock.poll(now) {
335            debug_assert!(unsafe { entry.is_pending() });
336
337            // SAFETY: We hold the driver lock, and just removed the entry from any linked lists.
338            if let Some(waker) = unsafe { entry.fire(Ok(())) } {
339                waker_list.push(waker);
340
341                if !waker_list.can_push() {
342                    // Wake a batch of wakers. To avoid deadlock, we must do this with the lock temporarily dropped.
343                    drop(lock);
344                    drop(wheels_lock);
345
346                    waker_list.wake_all();
347
348                    wheels_lock = self.inner.wheels.read();
349                    lock = wheels_lock.lock_sharded_wheel(id);
350                }
351            }
352        }
353        let next_wake_up = lock.poll_at();
354        drop(lock);
355        drop(wheels_lock);
356
357        waker_list.wake_all();
358        next_wake_up
359    }
360
361    /// Removes a registered timer from the driver.
362    ///
363    /// The timer will be moved to the cancelled state. Wakers will _not_ be
364    /// invoked. If the timer is already completed, this function is a no-op.
365    ///
366    /// This function always acquires the driver lock, even if the entry does
367    /// not appear to be registered.
368    ///
369    /// SAFETY: The timer must not be registered with some other driver, and
370    /// `add_entry` must not be called concurrently.
371    pub(self) unsafe fn clear_entry(&self, entry: NonNull<TimerShared>) {
372        unsafe {
373            let wheels_lock = self.inner.wheels.read();
374            let mut lock = wheels_lock.lock_sharded_wheel(entry.as_ref().shard_id());
375
376            if entry.as_ref().might_be_registered() {
377                lock.remove(entry);
378            }
379
380            entry.as_ref().handle().fire(Ok(()));
381        }
382    }
383
384    /// Removes and re-adds an entry to the driver.
385    ///
386    /// SAFETY: The timer must be either unregistered, or registered with this
387    /// driver. No other threads are allowed to concurrently manipulate the
388    /// timer at all (the current thread should hold an exclusive reference to
389    /// the `TimerEntry`)
390    pub(self) unsafe fn reregister(
391        &self,
392        unpark: &IoHandle,
393        new_tick: u64,
394        entry: NonNull<TimerShared>,
395    ) {
396        let waker = unsafe {
397            let wheels_lock = self.inner.wheels.read();
398
399            let mut lock = wheels_lock.lock_sharded_wheel(entry.as_ref().shard_id());
400
401            // We may have raced with a firing/deregistration, so check before
402            // deregistering.
403            if unsafe { entry.as_ref().might_be_registered() } {
404                lock.remove(entry);
405            }
406
407            // Now that we have exclusive control of this entry, mint a handle to reinsert it.
408            let entry = entry.as_ref().handle();
409
410            if self.is_shutdown() {
411                unsafe { entry.fire(Err(crate::time::error::Error::shutdown())) }
412            } else {
413                entry.set_expiration(new_tick);
414
415                // Note: We don't have to worry about racing with some other resetting
416                // thread, because add_entry and reregister require exclusive control of
417                // the timer entry.
418                match unsafe { lock.insert(entry) } {
419                    Ok(when) => {
420                        if self
421                            .inner
422                            .next_wake
423                            .load()
424                            .map(|next_wake| when < next_wake.get())
425                            .unwrap_or(true)
426                        {
427                            unpark.unpark();
428                        }
429
430                        None
431                    }
432                    Err((entry, crate::time::error::InsertError::Elapsed)) => unsafe {
433                        entry.fire(Ok(()))
434                    },
435                }
436            }
437
438            // Must release lock before invoking waker to avoid the risk of deadlock.
439        };
440
441        // The timer was fired synchronously as a result of the reregistration.
442        // Wake the waker; this is needed because we might reset _after_ a poll,
443        // and otherwise the task won't be awoken to poll again.
444        if let Some(waker) = waker {
445            waker.wake();
446        }
447    }
448
449    cfg_test_util! {
450        fn did_wake(&self) -> bool {
451            self.inner.did_wake.swap(false, Ordering::SeqCst)
452        }
453    }
454}
455
456// ===== impl Inner =====
457
458impl Inner {
459    // Check whether the driver has been shutdown
460    pub(super) fn is_shutdown(&self) -> bool {
461        self.is_shutdown.load(Ordering::SeqCst)
462    }
463
464    // Gets the number of shards.
465    fn get_shard_size(&self) -> u32 {
466        self.wheels_len
467    }
468}
469
470impl fmt::Debug for Inner {
471    fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result {
472        fmt.debug_struct("Inner").finish()
473    }
474}
475
476// ===== impl ShardedWheel =====
477
478impl ShardedWheel {
479    /// Locks the driver's sharded wheel structure.
480    pub(super) fn lock_sharded_wheel(
481        &self,
482        shard_id: u32,
483    ) -> crate::loom::sync::MutexGuard<'_, Wheel> {
484        let index = shard_id % (self.0.len() as u32);
485        // Safety: This modulo operation ensures that the index is not out of bounds.
486        unsafe { self.0.get_unchecked(index as usize) }.lock()
487    }
488}
489
490#[cfg(test)]
491mod tests;