tokio_reactor/
lib.rs

1#![doc(html_root_url = "https://docs.rs/tokio-reactor/0.1.12")]
2#![deny(missing_docs, missing_debug_implementations)]
3
4//! Event loop that drives Tokio I/O resources.
5//!
6//! > **Note:** This crate is **deprecated in tokio 0.2.x** and has been moved
7//! > and refactored into various places in the [`tokio::runtime`] and
8//! > [`tokio::io`] modules of the [`tokio`] crate. The Reactor has also been
9//! > renamed the "I/O Driver".
10//!
11//! [`tokio::runtime`]: https://docs.rs/tokio/latest/tokio/runtime/index.html
12//! [`tokio::io`]: https://docs.rs/tokio/latest/tokio/io/index.html
13//! [`tokio`]: https://docs.rs/tokio/latest/tokio/index.html
14//! [`io-driver` feature]: https://docs.rs/tokio/0.2.9/tokio/index.html#feature-flags
15//!
16//! The reactor is the engine that drives asynchronous I/O resources (like TCP and
17//! UDP sockets). It is backed by [`mio`] and acts as a bridge between [`mio`] and
18//! [`futures`].
19//!
20//! The crate provides:
21//!
22//! * [`Reactor`] is the main type of this crate. It performs the event loop logic.
23//!
24//! * [`Handle`] provides a reference to a reactor instance.
25//!
26//! * [`Registration`] and [`PollEvented`] allow third parties to implement I/O
27//!   resources that are driven by the reactor.
28//!
29//! Application authors will not use this crate directly. Instead, they will use the
30//! `tokio` crate. Library authors should only depend on `tokio-reactor` if they
31//! are building a custom I/O resource.
32//!
33//! For more details, see [reactor module] documentation in the Tokio crate.
34//!
35//! [`mio`]: http://github.com/carllerche/mio
36//! [`futures`]: http://github.com/rust-lang-nursery/futures-rs
37//! [`Reactor`]: struct.Reactor.html
38//! [`Handle`]: struct.Handle.html
39//! [`Registration`]: struct.Registration.html
40//! [`PollEvented`]: struct.PollEvented.html
41//! [reactor module]: https://docs.rs/tokio/0.1/tokio/reactor/index.html
42
43extern crate crossbeam_utils;
44#[macro_use]
45extern crate futures;
46#[macro_use]
47extern crate lazy_static;
48#[macro_use]
49extern crate log;
50extern crate mio;
51extern crate num_cpus;
52extern crate parking_lot;
53extern crate slab;
54extern crate tokio_executor;
55extern crate tokio_io;
56extern crate tokio_sync;
57
58pub(crate) mod background;
59mod poll_evented;
60mod registration;
61mod sharded_rwlock;
62
63// ===== Public re-exports =====
64
65pub use self::background::{Background, Shutdown};
66pub use self::poll_evented::PollEvented;
67pub use self::registration::Registration;
68
69// ===== Private imports =====
70
71use sharded_rwlock::RwLock;
72
73use futures::task::Task;
74use tokio_executor::park::{Park, Unpark};
75use tokio_executor::Enter;
76use tokio_sync::task::AtomicTask;
77
78use std::cell::RefCell;
79use std::error::Error;
80use std::io;
81use std::mem;
82#[cfg(all(unix, not(target_os = "fuchsia")))]
83use std::os::unix::io::{AsRawFd, RawFd};
84use std::sync::atomic::AtomicUsize;
85use std::sync::atomic::Ordering::{Relaxed, SeqCst};
86use std::sync::{Arc, Weak};
87use std::time::{Duration, Instant};
88use std::{fmt, usize};
89
90use log::Level;
91use mio::event::Evented;
92use slab::Slab;
93
94/// The core reactor, or event loop.
95///
96/// The event loop is the main source of blocking in an application which drives
97/// all other I/O events and notifications happening. Each event loop can have
98/// multiple handles pointing to it, each of which can then be used to create
99/// various I/O objects to interact with the event loop in interesting ways.
100pub struct Reactor {
101    /// Reuse the `mio::Events` value across calls to poll.
102    events: mio::Events,
103
104    /// State shared between the reactor and the handles.
105    inner: Arc<Inner>,
106
107    _wakeup_registration: mio::Registration,
108}
109
110/// A reference to a reactor.
111///
112/// A `Handle` is used for associating I/O objects with an event loop
113/// explicitly. Typically though you won't end up using a `Handle` that often
114/// and will instead use the default reactor for the execution context.
115///
116/// By default, most components bind lazily to reactors.
117/// To get this behavior when manually passing a `Handle`, use `default()`.
118#[derive(Clone)]
119pub struct Handle {
120    inner: Option<HandlePriv>,
121}
122
123/// Like `Handle`, but never `None`.
124#[derive(Clone)]
125struct HandlePriv {
126    inner: Weak<Inner>,
127}
128
129/// Return value from the `turn` method on `Reactor`.
130///
131/// Currently this value doesn't actually provide any functionality, but it may
132/// in the future give insight into what happened during `turn`.
133#[derive(Debug)]
134pub struct Turn {
135    _priv: (),
136}
137
138/// Error returned from `Handle::set_fallback`.
139#[derive(Clone, Debug)]
140pub struct SetFallbackError(());
141
142#[deprecated(since = "0.1.2", note = "use SetFallbackError instead")]
143#[doc(hidden)]
144pub type SetDefaultError = SetFallbackError;
145
146/// Ensure that the default reactor is removed from the thread-local context
147/// when leaving the scope. This handles cases that involve panicking.
148#[derive(Debug)]
149pub struct DefaultGuard {
150    _p: (),
151}
152
153#[test]
154fn test_handle_size() {
155    use std::mem;
156    assert_eq!(mem::size_of::<Handle>(), mem::size_of::<HandlePriv>());
157}
158
159struct Inner {
160    /// The underlying system event queue.
161    io: mio::Poll,
162
163    /// ABA guard counter
164    next_aba_guard: AtomicUsize,
165
166    /// Dispatch slabs for I/O and futures events
167    io_dispatch: RwLock<Slab<ScheduledIo>>,
168
169    /// Used to wake up the reactor from a call to `turn`
170    wakeup: mio::SetReadiness,
171}
172
173struct ScheduledIo {
174    aba_guard: usize,
175    readiness: AtomicUsize,
176    reader: AtomicTask,
177    writer: AtomicTask,
178}
179
180#[derive(Debug, Eq, PartialEq, Clone, Copy)]
181pub(crate) enum Direction {
182    Read,
183    Write,
184}
185
186/// The global fallback reactor.
187static HANDLE_FALLBACK: AtomicUsize = AtomicUsize::new(0);
188
189thread_local! {
190    /// Tracks the reactor for the current execution context.
191    static CURRENT_REACTOR: RefCell<Option<HandlePriv>> = RefCell::new(None)
192}
193
194const TOKEN_SHIFT: usize = 22;
195
196// Kind of arbitrary, but this reserves some token space for later usage.
197const MAX_SOURCES: usize = (1 << TOKEN_SHIFT) - 1;
198const TOKEN_WAKEUP: mio::Token = mio::Token(MAX_SOURCES);
199
200fn _assert_kinds() {
201    fn _assert<T: Send + Sync>() {}
202
203    _assert::<Handle>();
204}
205
206// ===== impl Reactor =====
207
208/// Set the default reactor for the duration of the closure
209///
210/// # Panics
211///
212/// This function panics if there already is a default reactor set.
213pub fn with_default<F, R>(handle: &Handle, enter: &mut Enter, f: F) -> R
214where
215    F: FnOnce(&mut Enter) -> R,
216{
217    // This ensures the value for the current reactor gets reset even if there
218    // is a panic.
219    let _guard = set_default(handle);
220    f(enter)
221}
222
223/// Sets `handle` as the default reactor, returning a guard that unsets it when
224/// dropped.
225///
226/// # Panics
227///
228/// This function panics if there already is a default reactor set.
229pub fn set_default(handle: &Handle) -> DefaultGuard {
230    CURRENT_REACTOR.with(|current| {
231        let mut current = current.borrow_mut();
232
233        assert!(
234            current.is_none(),
235            "default Tokio reactor already set \
236             for execution context"
237        );
238
239        let handle = match handle.as_priv() {
240            Some(handle) => handle,
241            None => {
242                panic!("`handle` does not reference a reactor");
243            }
244        };
245
246        *current = Some(handle.clone());
247    });
248    DefaultGuard { _p: () }
249}
250
251impl Reactor {
252    /// Creates a new event loop, returning any error that happened during the
253    /// creation.
254    pub fn new() -> io::Result<Reactor> {
255        let io = mio::Poll::new()?;
256        let wakeup_pair = mio::Registration::new2();
257
258        io.register(
259            &wakeup_pair.0,
260            TOKEN_WAKEUP,
261            mio::Ready::readable(),
262            mio::PollOpt::level(),
263        )?;
264
265        Ok(Reactor {
266            events: mio::Events::with_capacity(1024),
267            _wakeup_registration: wakeup_pair.0,
268            inner: Arc::new(Inner {
269                io: io,
270                next_aba_guard: AtomicUsize::new(0),
271                io_dispatch: RwLock::new(Slab::with_capacity(1)),
272                wakeup: wakeup_pair.1,
273            }),
274        })
275    }
276
277    /// Returns a handle to this event loop which can be sent across threads
278    /// and can be used as a proxy to the event loop itself.
279    ///
280    /// Handles are cloneable and clones always refer to the same event loop.
281    /// This handle is typically passed into functions that create I/O objects
282    /// to bind them to this event loop.
283    pub fn handle(&self) -> Handle {
284        Handle {
285            inner: Some(HandlePriv {
286                inner: Arc::downgrade(&self.inner),
287            }),
288        }
289    }
290
291    /// Configures the fallback handle to be returned from `Handle::default`.
292    ///
293    /// The `Handle::default()` function will by default lazily spin up a global
294    /// thread and run a reactor on this global thread. This behavior is not
295    /// always desirable in all applications, however, and sometimes a different
296    /// fallback reactor is desired.
297    ///
298    /// This function will attempt to globally alter the return value of
299    /// `Handle::default()` to return the `handle` specified rather than a
300    /// lazily initialized global thread. If successful then all future calls to
301    /// `Handle::default()` which would otherwise fall back to the global thread
302    /// will instead return a clone of the handle specified.
303    ///
304    /// # Errors
305    ///
306    /// This function may not always succeed in configuring the fallback handle.
307    /// If this function was previously called (or perhaps concurrently called
308    /// on many threads) only the *first* invocation of this function will
309    /// succeed. All other invocations will return an error.
310    ///
311    /// Additionally if the global reactor thread has already been initialized
312    /// then this function will also return an error. (aka if `Handle::default`
313    /// has been called previously in this program).
314    pub fn set_fallback(&self) -> Result<(), SetFallbackError> {
315        set_fallback(self.handle().into_priv().unwrap())
316    }
317
318    /// Performs one iteration of the event loop, blocking on waiting for events
319    /// for at most `max_wait` (forever if `None`).
320    ///
321    /// This method is the primary method of running this reactor and processing
322    /// I/O events that occur. This method executes one iteration of an event
323    /// loop, blocking at most once waiting for events to happen.
324    ///
325    /// If a `max_wait` is specified then the method should block no longer than
326    /// the duration specified, but this shouldn't be used as a super-precise
327    /// timer but rather a "ballpark approximation"
328    ///
329    /// # Return value
330    ///
331    /// This function returns an instance of `Turn`
332    ///
333    /// `Turn` as of today has no extra information with it and can be safely
334    /// discarded.  In the future `Turn` may contain information about what
335    /// happened while this reactor blocked.
336    ///
337    /// # Errors
338    ///
339    /// This function may also return any I/O error which occurs when polling
340    /// for readiness of I/O objects with the OS. This is quite unlikely to
341    /// arise and typically mean that things have gone horribly wrong at that
342    /// point. Currently this is primarily only known to happen for internal
343    /// bugs to `tokio` itself.
344    pub fn turn(&mut self, max_wait: Option<Duration>) -> io::Result<Turn> {
345        self.poll(max_wait)?;
346        Ok(Turn { _priv: () })
347    }
348
349    /// Returns true if the reactor is currently idle.
350    ///
351    /// Idle is defined as all tasks that have been spawned have completed,
352    /// either successfully or with an error.
353    pub fn is_idle(&self) -> bool {
354        self.inner.io_dispatch.read().is_empty()
355    }
356
357    /// Run this reactor on a background thread.
358    ///
359    /// This function takes ownership, spawns a new thread, and moves the
360    /// reactor to this new thread. It then runs the reactor, driving all
361    /// associated I/O resources, until the `Background` handle is dropped or
362    /// explicitly shutdown.
363    pub fn background(self) -> io::Result<Background> {
364        Background::new(self)
365    }
366
367    fn poll(&mut self, max_wait: Option<Duration>) -> io::Result<()> {
368        // Block waiting for an event to happen, peeling out how many events
369        // happened.
370        match self.inner.io.poll(&mut self.events, max_wait) {
371            Ok(_) => {}
372            Err(e) => return Err(e),
373        }
374
375        let start = if log_enabled!(Level::Debug) {
376            Some(Instant::now())
377        } else {
378            None
379        };
380
381        // Process all the events that came in, dispatching appropriately
382        let mut events = 0;
383        for event in self.events.iter() {
384            events += 1;
385            let token = event.token();
386            trace!("event {:?} {:?}", event.readiness(), event.token());
387
388            if token == TOKEN_WAKEUP {
389                self.inner
390                    .wakeup
391                    .set_readiness(mio::Ready::empty())
392                    .unwrap();
393            } else {
394                self.dispatch(token, event.readiness());
395            }
396        }
397
398        if let Some(start) = start {
399            let dur = start.elapsed();
400            trace!(
401                "loop process - {} events, {}.{:03}s",
402                events,
403                dur.as_secs(),
404                dur.subsec_nanos() / 1_000_000
405            );
406        }
407
408        Ok(())
409    }
410
411    fn dispatch(&self, token: mio::Token, ready: mio::Ready) {
412        let aba_guard = token.0 & !MAX_SOURCES;
413        let token = token.0 & MAX_SOURCES;
414
415        let mut rd = None;
416        let mut wr = None;
417
418        // Create a scope to ensure that notifying the tasks stays out of the
419        // lock's critical section.
420        {
421            let io_dispatch = self.inner.io_dispatch.read();
422
423            let io = match io_dispatch.get(token) {
424                Some(io) => io,
425                None => return,
426            };
427
428            if aba_guard != io.aba_guard {
429                return;
430            }
431
432            io.readiness.fetch_or(ready.as_usize(), Relaxed);
433
434            if ready.is_writable() || platform::is_hup(&ready) {
435                wr = io.writer.take_task();
436            }
437
438            if !(ready & (!mio::Ready::writable())).is_empty() {
439                rd = io.reader.take_task();
440            }
441        }
442
443        if let Some(task) = rd {
444            task.notify();
445        }
446
447        if let Some(task) = wr {
448            task.notify();
449        }
450    }
451}
452
453#[cfg(all(unix, not(target_os = "fuchsia")))]
454impl AsRawFd for Reactor {
455    fn as_raw_fd(&self) -> RawFd {
456        self.inner.io.as_raw_fd()
457    }
458}
459
460impl Park for Reactor {
461    type Unpark = Handle;
462    type Error = io::Error;
463
464    fn unpark(&self) -> Self::Unpark {
465        self.handle()
466    }
467
468    fn park(&mut self) -> io::Result<()> {
469        self.turn(None)?;
470        Ok(())
471    }
472
473    fn park_timeout(&mut self, duration: Duration) -> io::Result<()> {
474        self.turn(Some(duration))?;
475        Ok(())
476    }
477}
478
479impl fmt::Debug for Reactor {
480    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
481        write!(f, "Reactor")
482    }
483}
484
485// ===== impl Handle =====
486
487impl Handle {
488    #[doc(hidden)]
489    #[deprecated(note = "semantics were sometimes surprising, use Handle::default()")]
490    pub fn current() -> Handle {
491        // TODO: Should this panic on error?
492        HandlePriv::try_current()
493            .map(|handle| Handle {
494                inner: Some(handle),
495            })
496            .unwrap_or(Handle {
497                inner: Some(HandlePriv { inner: Weak::new() }),
498            })
499    }
500
501    fn as_priv(&self) -> Option<&HandlePriv> {
502        self.inner.as_ref()
503    }
504
505    fn into_priv(self) -> Option<HandlePriv> {
506        self.inner
507    }
508
509    fn wakeup(&self) {
510        if let Some(handle) = self.as_priv() {
511            handle.wakeup();
512        }
513    }
514}
515
516impl Unpark for Handle {
517    fn unpark(&self) {
518        if let Some(ref h) = self.inner {
519            h.wakeup();
520        }
521    }
522}
523
524impl Default for Handle {
525    /// Returns a "default" handle, i.e., a handle that lazily binds to a reactor.
526    fn default() -> Handle {
527        Handle { inner: None }
528    }
529}
530
531impl fmt::Debug for Handle {
532    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
533        write!(f, "Handle")
534    }
535}
536
537fn set_fallback(handle: HandlePriv) -> Result<(), SetFallbackError> {
538    unsafe {
539        let val = handle.into_usize();
540        match HANDLE_FALLBACK.compare_exchange(0, val, SeqCst, SeqCst) {
541            Ok(_) => Ok(()),
542            Err(_) => {
543                drop(HandlePriv::from_usize(val));
544                Err(SetFallbackError(()))
545            }
546        }
547    }
548}
549
550// ===== impl HandlePriv =====
551
552impl HandlePriv {
553    /// Try to get a handle to the current reactor.
554    ///
555    /// Returns `Err` if no handle is found.
556    pub(crate) fn try_current() -> io::Result<HandlePriv> {
557        CURRENT_REACTOR.with(|current| match *current.borrow() {
558            Some(ref handle) => Ok(handle.clone()),
559            None => HandlePriv::fallback(),
560        })
561    }
562
563    /// Returns a handle to the fallback reactor.
564    fn fallback() -> io::Result<HandlePriv> {
565        let mut fallback = HANDLE_FALLBACK.load(SeqCst);
566
567        // If the fallback hasn't been previously initialized then let's spin
568        // up a helper thread and try to initialize with that. If we can't
569        // actually create a helper thread then we'll just return a "defunct"
570        // handle which will return errors when I/O objects are attempted to be
571        // associated.
572        if fallback == 0 {
573            let reactor = match Reactor::new() {
574                Ok(reactor) => reactor,
575                Err(_) => {
576                    return Err(io::Error::new(
577                        io::ErrorKind::Other,
578                        "failed to create reactor",
579                    ));
580                }
581            };
582
583            // If we successfully set ourselves as the actual fallback then we
584            // want to `forget` the helper thread to ensure that it persists
585            // globally. If we fail to set ourselves as the fallback that means
586            // that someone was racing with this call to `Handle::default`.
587            // They ended up winning so we'll destroy our helper thread (which
588            // shuts down the thread) and reload the fallback.
589            if set_fallback(reactor.handle().into_priv().unwrap()).is_ok() {
590                let ret = reactor.handle().into_priv().unwrap();
591
592                match reactor.background() {
593                    Ok(bg) => bg.forget(),
594                    // The global handle is fubar, but y'all probably got bigger
595                    // problems if a thread can't spawn.
596                    Err(_) => {}
597                }
598
599                return Ok(ret);
600            }
601
602            fallback = HANDLE_FALLBACK.load(SeqCst);
603        }
604
605        // At this point our fallback handle global was configured so we use
606        // its value to reify a handle, clone it, and then forget our reified
607        // handle as we don't actually have an owning reference to it.
608        assert!(fallback != 0);
609
610        let ret = unsafe {
611            let handle = HandlePriv::from_usize(fallback);
612            let ret = handle.clone();
613
614            // This prevents `handle` from being dropped and having the ref
615            // count decremented.
616            drop(handle.into_usize());
617
618            ret
619        };
620
621        Ok(ret)
622    }
623
624    /// Forces a reactor blocked in a call to `turn` to wakeup, or otherwise
625    /// makes the next call to `turn` return immediately.
626    ///
627    /// This method is intended to be used in situations where a notification
628    /// needs to otherwise be sent to the main reactor. If the reactor is
629    /// currently blocked inside of `turn` then it will wake up and soon return
630    /// after this method has been called. If the reactor is not currently
631    /// blocked in `turn`, then the next call to `turn` will not block and
632    /// return immediately.
633    fn wakeup(&self) {
634        if let Some(inner) = self.inner() {
635            inner.wakeup.set_readiness(mio::Ready::readable()).unwrap();
636        }
637    }
638
639    fn into_usize(self) -> usize {
640        unsafe { mem::transmute::<Weak<Inner>, usize>(self.inner) }
641    }
642
643    unsafe fn from_usize(val: usize) -> HandlePriv {
644        let inner = mem::transmute::<usize, Weak<Inner>>(val);
645        HandlePriv { inner }
646    }
647
648    fn inner(&self) -> Option<Arc<Inner>> {
649        self.inner.upgrade()
650    }
651}
652
653impl fmt::Debug for HandlePriv {
654    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
655        write!(f, "HandlePriv")
656    }
657}
658
659// ===== impl Inner =====
660
661impl Inner {
662    /// Register an I/O resource with the reactor.
663    ///
664    /// The registration token is returned.
665    fn add_source(&self, source: &dyn Evented) -> io::Result<usize> {
666        // Get an ABA guard value
667        let aba_guard = self.next_aba_guard.fetch_add(1 << TOKEN_SHIFT, Relaxed);
668
669        let key = {
670            // Block to contain the write lock
671            let mut io_dispatch = self.io_dispatch.write();
672
673            if io_dispatch.len() == MAX_SOURCES {
674                return Err(io::Error::new(
675                    io::ErrorKind::Other,
676                    "reactor at max \
677                     registered I/O resources",
678                ));
679            }
680
681            io_dispatch.insert(ScheduledIo {
682                aba_guard,
683                readiness: AtomicUsize::new(0),
684                reader: AtomicTask::new(),
685                writer: AtomicTask::new(),
686            })
687        };
688
689        let token = aba_guard | key;
690        debug!("adding I/O source: {}", token);
691
692        self.io.register(
693            source,
694            mio::Token(token),
695            mio::Ready::all(),
696            mio::PollOpt::edge(),
697        )?;
698
699        Ok(key)
700    }
701
702    /// Deregisters an I/O resource from the reactor.
703    fn deregister_source(&self, source: &dyn Evented) -> io::Result<()> {
704        self.io.deregister(source)
705    }
706
707    fn drop_source(&self, token: usize) {
708        debug!("dropping I/O source: {}", token);
709        self.io_dispatch.write().remove(token);
710    }
711
712    /// Registers interest in the I/O resource associated with `token`.
713    fn register(&self, token: usize, dir: Direction, t: Task) {
714        debug!("scheduling {:?} for: {}", dir, token);
715        let io_dispatch = self.io_dispatch.read();
716        let sched = io_dispatch.get(token).unwrap();
717
718        let (task, ready) = match dir {
719            Direction::Read => (&sched.reader, !mio::Ready::writable()),
720            Direction::Write => (&sched.writer, mio::Ready::writable()),
721        };
722
723        task.register_task(t);
724
725        if sched.readiness.load(SeqCst) & ready.as_usize() != 0 {
726            task.notify();
727        }
728    }
729}
730
731impl Drop for Inner {
732    fn drop(&mut self) {
733        // When a reactor is dropped it needs to wake up all blocked tasks as
734        // they'll never receive a notification, and all connected I/O objects
735        // will start returning errors pretty quickly.
736        let io = self.io_dispatch.read();
737        for (_, io) in io.iter() {
738            io.writer.notify();
739            io.reader.notify();
740        }
741    }
742}
743
744impl Direction {
745    fn mask(&self) -> mio::Ready {
746        match *self {
747            Direction::Read => {
748                // Everything except writable is signaled through read.
749                mio::Ready::all() - mio::Ready::writable()
750            }
751            Direction::Write => mio::Ready::writable() | platform::hup(),
752        }
753    }
754}
755
756impl Drop for DefaultGuard {
757    fn drop(&mut self) {
758        let _ = CURRENT_REACTOR.try_with(|current| {
759            let mut current = current.borrow_mut();
760            *current = None;
761        });
762    }
763}
764
765#[cfg(unix)]
766mod platform {
767    use mio::unix::UnixReady;
768    use mio::Ready;
769
770    pub fn hup() -> Ready {
771        UnixReady::hup().into()
772    }
773
774    pub fn is_hup(ready: &Ready) -> bool {
775        UnixReady::from(*ready).is_hup()
776    }
777}
778
779#[cfg(windows)]
780mod platform {
781    use mio::Ready;
782
783    pub fn hup() -> Ready {
784        Ready::empty()
785    }
786
787    pub fn is_hup(_: &Ready) -> bool {
788        false
789    }
790}
791
792// ===== impl SetFallbackError =====
793
794impl fmt::Display for SetFallbackError {
795    fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
796        write!(fmt, "{}", self.description())
797    }
798}
799
800impl Error for SetFallbackError {
801    fn description(&self) -> &str {
802        "attempted to set fallback reactor while already configured"
803    }
804}