summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorCarl Lerche <me@carllerche.com>2019-11-05 19:12:30 -0800
committerGitHub <noreply@github.com>2019-11-05 19:12:30 -0800
commitd5c1119c881c9a8b511aa9000fd26b9bda014256 (patch)
tree72e2ca6b655f29e948a91ba4573a95350cb241e0
parenta6253ed05a1e0d14bc64915f5937c29092df9497 (diff)
runtime: combine `executor` and `runtime` mods (#1734)
Now, all types are under `runtime`. `executor::util` is moved to a top level `util` module.
-rw-r--r--tokio-test/src/clock.rs2
-rw-r--r--tokio/src/executor/mod.rs74
-rw-r--r--tokio/src/fs/mod.rs2
-rw-r--r--tokio/src/lib.rs11
-rw-r--r--tokio/src/loom/std/mod.rs1
-rw-r--r--tokio/src/net/addr.rs4
-rw-r--r--tokio/src/net/driver/reactor/mod.rs2
-rw-r--r--tokio/src/runtime/blocking/builder.rs (renamed from tokio/src/executor/blocking/builder.rs)2
-rw-r--r--tokio/src/runtime/blocking/mod.rs (renamed from tokio/src/executor/blocking/mod.rs)8
-rw-r--r--tokio/src/runtime/builder.rs114
-rw-r--r--tokio/src/runtime/current_thread/mod.rs (renamed from tokio/src/executor/current_thread/mod.rs)11
-rw-r--r--tokio/src/runtime/enter.rs (renamed from tokio/src/executor/enter.rs)6
-rw-r--r--tokio/src/runtime/global.rs (renamed from tokio/src/executor/global.rs)4
-rw-r--r--tokio/src/runtime/io.rs56
-rw-r--r--tokio/src/runtime/mod.rs96
-rw-r--r--tokio/src/runtime/park/mod.rs (renamed from tokio/src/executor/park/mod.rs)5
-rw-r--r--tokio/src/runtime/park/thread.rs (renamed from tokio/src/executor/park/thread.rs)60
-rw-r--r--tokio/src/runtime/spawner.rs20
-rw-r--r--tokio/src/runtime/task/core.rs (renamed from tokio/src/executor/task/core.rs)8
-rw-r--r--tokio/src/runtime/task/error.rs (renamed from tokio/src/executor/task/error.rs)0
-rw-r--r--tokio/src/runtime/task/harness.rs (renamed from tokio/src/executor/task/harness.rs)6
-rw-r--r--tokio/src/runtime/task/join.rs (renamed from tokio/src/executor/task/join.rs)2
-rw-r--r--tokio/src/runtime/task/list.rs (renamed from tokio/src/executor/task/list.rs)2
-rw-r--r--tokio/src/runtime/task/mod.rs (renamed from tokio/src/executor/task/mod.rs)7
-rw-r--r--tokio/src/runtime/task/raw.rs (renamed from tokio/src/executor/task/raw.rs)8
-rw-r--r--tokio/src/runtime/task/stack.rs (renamed from tokio/src/executor/task/stack.rs)2
-rw-r--r--tokio/src/runtime/task/state.rs (renamed from tokio/src/executor/task/state.rs)0
-rw-r--r--tokio/src/runtime/task/tests/loom.rs (renamed from tokio/src/executor/task/tests/loom.rs)4
-rw-r--r--tokio/src/runtime/task/tests/mod.rs (renamed from tokio/src/executor/task/tests/mod.rs)0
-rw-r--r--tokio/src/runtime/task/tests/task.rs (renamed from tokio/src/executor/task/tests/task.rs)8
-rw-r--r--tokio/src/runtime/task/waker.rs (renamed from tokio/src/executor/task/waker.rs)4
-rw-r--r--tokio/src/runtime/tests/backoff.rs (renamed from tokio/src/executor/tests/backoff.rs)0
-rw-r--r--tokio/src/runtime/tests/loom_oneshot.rs (renamed from tokio/src/executor/tests/loom_oneshot.rs)0
-rw-r--r--tokio/src/runtime/tests/loom_schedule.rs (renamed from tokio/src/executor/tests/loom_schedule.rs)2
-rw-r--r--tokio/src/runtime/tests/mock_park.rs (renamed from tokio/src/executor/tests/mock_park.rs)2
-rw-r--r--tokio/src/runtime/tests/mock_schedule.rs (renamed from tokio/src/executor/tests/mock_schedule.rs)2
-rw-r--r--tokio/src/runtime/tests/mod.rs (renamed from tokio/src/executor/tests/mod.rs)0
-rw-r--r--tokio/src/runtime/tests/track_drop.rs (renamed from tokio/src/executor/tests/track_drop.rs)0
-rw-r--r--tokio/src/runtime/thread_pool/builder.rs (renamed from tokio/src/executor/thread_pool/builder.rs)14
-rw-r--r--tokio/src/runtime/thread_pool/current.rs (renamed from tokio/src/executor/thread_pool/current.rs)4
-rw-r--r--tokio/src/runtime/thread_pool/idle.rs (renamed from tokio/src/executor/thread_pool/idle.rs)0
-rw-r--r--tokio/src/runtime/thread_pool/mod.rs (renamed from tokio/src/executor/thread_pool/mod.rs)2
-rw-r--r--tokio/src/runtime/thread_pool/owned.rs (renamed from tokio/src/executor/thread_pool/owned.rs)6
-rw-r--r--tokio/src/runtime/thread_pool/pool.rs (renamed from tokio/src/executor/thread_pool/pool.rs)14
-rw-r--r--tokio/src/runtime/thread_pool/queue/global.rs (renamed from tokio/src/executor/thread_pool/queue/global.rs)2
-rw-r--r--tokio/src/runtime/thread_pool/queue/inject.rs (renamed from tokio/src/executor/thread_pool/queue/inject.rs)4
-rw-r--r--tokio/src/runtime/thread_pool/queue/local.rs (renamed from tokio/src/executor/thread_pool/queue/local.rs)6
-rw-r--r--tokio/src/runtime/thread_pool/queue/mod.rs (renamed from tokio/src/executor/thread_pool/queue/mod.rs)0
-rw-r--r--tokio/src/runtime/thread_pool/queue/worker.rs (renamed from tokio/src/executor/thread_pool/queue/worker.rs)4
-rw-r--r--tokio/src/runtime/thread_pool/set.rs (renamed from tokio/src/executor/thread_pool/set.rs)14
-rw-r--r--tokio/src/runtime/thread_pool/shared.rs (renamed from tokio/src/executor/thread_pool/shared.rs)6
-rw-r--r--tokio/src/runtime/thread_pool/shutdown.rs (renamed from tokio/src/executor/thread_pool/shutdown.rs)2
-rw-r--r--tokio/src/runtime/thread_pool/spawner.rs (renamed from tokio/src/executor/thread_pool/spawner.rs)8
-rw-r--r--tokio/src/runtime/thread_pool/tests/loom_pool.rs (renamed from tokio/src/executor/thread_pool/tests/loom_pool.rs)6
-rw-r--r--tokio/src/runtime/thread_pool/tests/loom_queue.rs (renamed from tokio/src/executor/thread_pool/tests/loom_queue.rs)6
-rw-r--r--tokio/src/runtime/thread_pool/tests/mod.rs (renamed from tokio/src/executor/thread_pool/tests/mod.rs)0
-rw-r--r--tokio/src/runtime/thread_pool/tests/pool.rs (renamed from tokio/src/executor/thread_pool/tests/pool.rs)3
-rw-r--r--tokio/src/runtime/thread_pool/tests/queue.rs (renamed from tokio/src/executor/thread_pool/tests/queue.rs)6
-rw-r--r--tokio/src/runtime/thread_pool/tests/worker.rs (renamed from tokio/src/executor/thread_pool/tests/worker.rs)8
-rw-r--r--tokio/src/runtime/thread_pool/worker.rs (renamed from tokio/src/executor/thread_pool/worker.rs)19
-rw-r--r--tokio/src/runtime/timer.rs41
-rw-r--r--tokio/src/timer/timer/mod.rs4
-rw-r--r--tokio/src/util/mod.rs (renamed from tokio/src/executor/util/mod.rs)4
-rw-r--r--tokio/src/util/pad.rs (renamed from tokio/src/executor/util/pad.rs)0
-rw-r--r--tokio/src/util/rand.rs (renamed from tokio/src/executor/util/rand.rs)0
-rw-r--r--tokio/tests/clock.rs6
-rw-r--r--tokio/tests/rt_thread_pool.rs6
67 files changed, 439 insertions, 291 deletions
diff --git a/tokio-test/src/clock.rs b/tokio-test/src/clock.rs
index c5fac787..38417539 100644
--- a/tokio-test/src/clock.rs
+++ b/tokio-test/src/clock.rs
@@ -22,7 +22,7 @@
//! });
//! ```
-use tokio::executor::park::{Park, Unpark};
+use tokio::runtime::{Park, Unpark};
use tokio::timer::clock::{Clock, Now};
use tokio::timer::Timer;
diff --git a/tokio/src/executor/mod.rs b/tokio/src/executor/mod.rs
deleted file mode 100644
index 6134ea3b..00000000
--- a/tokio/src/executor/mod.rs
+++ /dev/null
@@ -1,74 +0,0 @@
-//! Task execution related traits and utilities.
-//!
-//! In the Tokio execution model, futures are lazy. When a future is created, no
-//! work is performed. In order for the work defined by the future to happen,
-//! the future must be submitted to an executor. A future that is submitted to
-//! an executor is called a "task".
-//!
-//! The executor is responsible for ensuring that [`Future::poll`] is called
-//! whenever the task is notified. Notification happens when the internal
-//! state of a task transitions from *not ready* to *ready*. For example, a
-//! socket might have received data and a call to `read` will now be able to
-//! succeed.
-//!
-//! The specific strategy used to manage the tasks is left up to the
-//! executor. There are two main flavors of executors: single-threaded and
-//! multi-threaded. Tokio provides implementation for both of these in the
-//! [`runtime`] module.
-//!
-//! # `Executor` trait.
-//!
-//! This module provides the [`Executor`] trait (re-exported from
-//! [`tokio-executor`]), which describes the API that all executors must
-//! implement.
-//!
-//! A free [`spawn`] function is provided that allows spawning futures onto the
-//! default executor (tracked via a thread-local variable) without referencing a
-//! handle. It is expected that all executors will set a value for the default
-//! executor. This value will often be set to the executor itself, but it is
-//! possible that the default executor might be set to a different executor.
-//!
-//! For example, a single threaded executor might set the default executor to a
-//! thread pool instead of itself, allowing futures to spawn new tasks onto the
-//! thread pool when those tasks are `Send`.
-//!
-//! [`Future::poll`]: https://docs.rs/futures/0.1/futures/future/trait.Future.html#tymethod.poll
-//! [notified]: https://docs.rs/futures/0.1/futures/executor/trait.Notify.html#tymethod.notify
-//! [`runtime`]: ../runtime/index.html
-//! [`tokio-executor`]: https://docs.rs/tokio-executor/0.1
-//! [`Executor`]: trait.Executor.html
-//! [`spawn`]: fn.spawn.html#[cfg(all(test, loom))]
-
-// At the top due to macros
-#[cfg(test)]
-#[macro_use]
-mod tests;
-
-#[cfg(feature = "rt-current-thread")]
-mod enter;
-#[cfg(feature = "rt-current-thread")]
-pub(crate) use self::enter::enter;
-
-mod global;
-pub use self::global::spawn;
-
-pub mod park;
-
-#[cfg(feature = "rt-current-thread")]
-mod task;
-#[cfg(feature = "rt-current-thread")]
-pub use self::task::{JoinError, JoinHandle};
-
-#[cfg(feature = "rt-full")]
-mod util;
-
-#[cfg(all(not(feature = "blocking"), feature = "rt-full"))]
-mod blocking;
-#[cfg(feature = "blocking")]
-pub mod blocking;
-
-#[cfg(feature = "rt-current-thread")]
-pub(crate) mod current_thread;
-
-#[cfg(feature = "rt-full")]
-pub(crate) mod thread_pool;
diff --git a/tokio/src/fs/mod.rs b/tokio/src/fs/mod.rs
index c1b4ed71..ed3b4162 100644
--- a/tokio/src/fs/mod.rs
+++ b/tokio/src/fs/mod.rs
@@ -91,5 +91,5 @@ where
mod sys {
pub(crate) use std::fs::File;
- pub(crate) use crate::executor::blocking::{run, Blocking};
+ pub(crate) use crate::runtime::blocking::{run, Blocking};
}
diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs
index 89547c1d..b7b99ac6 100644
--- a/tokio/src/lib.rs
+++ b/tokio/src/lib.rs
@@ -72,6 +72,7 @@
macro_rules! if_runtime {
($($i:item)*) => ($(
#[cfg(any(
+ feature = "blocking",
feature = "rt-full",
feature = "rt-current-thread",
))]
@@ -98,7 +99,6 @@ pub mod io;
#[cfg(feature = "net-driver")]
pub mod net;
-#[cfg(any(feature = "sync", feature = "blocking", feature = "rt-current-thread"))]
mod loom;
pub mod prelude;
@@ -106,6 +106,8 @@ pub mod prelude;
#[cfg(all(feature = "process", not(loom)))]
pub mod process;
+pub mod runtime;
+
#[cfg(feature = "signal")]
#[cfg(not(loom))]
pub mod signal;
@@ -118,14 +120,13 @@ pub mod sync;
#[cfg(feature = "timer")]
pub mod timer;
-#[cfg(feature = "executor-core")]
-pub mod executor;
+#[cfg(feature = "rt-full")]
+mod util;
if_runtime! {
- pub mod runtime;
#[doc(inline)]
- pub use crate::executor::spawn;
+ pub use crate::runtime::spawn;
#[cfg(not(test))] // Work around for rust-lang/rust#62127
#[cfg(feature = "macros")]
diff --git a/tokio/src/loom/std/mod.rs b/tokio/src/loom/std/mod.rs
index 5bbf1531..e0aafa83 100644
--- a/tokio/src/loom/std/mod.rs
+++ b/tokio/src/loom/std/mod.rs
@@ -12,6 +12,7 @@ pub(crate) mod cell {
pub(crate) use super::causal_cell::{CausalCell, CausalCheck};
}
+#[cfg(feature = "sync")]
pub(crate) mod future {
pub(crate) use crate::sync::AtomicWaker;
}
diff --git a/tokio/src/net/addr.rs b/tokio/src/net/addr.rs
index 812964da..8fe02b4e 100644
--- a/tokio/src/net/addr.rs
+++ b/tokio/src/net/addr.rs
@@ -1,4 +1,4 @@
-use crate::executor::blocking;
+use crate::runtime::blocking;
use futures_util::future;
use std::io;
@@ -143,7 +143,7 @@ pub(crate) mod sealed {
//! part of the `ToSocketAddrs` public API. The details will change over
//! time.
- use crate::executor::blocking::Blocking;
+ use crate::runtime::blocking::Blocking;
use futures_core::ready;
use std::future::Future;
diff --git a/tokio/src/net/driver/reactor/mod.rs b/tokio/src/net/driver/reactor/mod.rs
index 3a2f3515..8bcded41 100644
--- a/tokio/src/net/driver/reactor/mod.rs
+++ b/tokio/src/net/driver/reactor/mod.rs
@@ -1,6 +1,6 @@
-use crate::executor::park::{Park, Unpark};
use crate::loom::sync::atomic::AtomicUsize;
use crate::net::driver::platform;
+use crate::runtime::{Park, Unpark};
use std::sync::atomic::Ordering::SeqCst;
diff --git a/tokio/src/executor/blocking/builder.rs b/tokio/src/runtime/blocking/builder.rs
index e755ae23..59178f25 100644
--- a/tokio/src/executor/blocking/builder.rs
+++ b/tokio/src/runtime/blocking/builder.rs
@@ -1,5 +1,5 @@
-use crate::executor::blocking::Pool;
use crate::loom::thread;
+use crate::runtime::blocking::Pool;
use std::usize;
diff --git a/tokio/src/executor/blocking/mod.rs b/tokio/src/runtime/blocking/mod.rs
index d105fe22..e634ea59 100644
--- a/tokio/src/executor/blocking/mod.rs
+++ b/tokio/src/runtime/blocking/mod.rs
@@ -262,7 +262,7 @@ impl Drop for PoolWaiter {
///
/// ```
/// # async fn docs() {
-/// tokio::executor::blocking::in_place(move || {
+/// tokio::runtime::blocking::in_place(move || {
/// // do some compute-heavy work or call synchronous code
/// });
/// # }
@@ -272,9 +272,9 @@ pub fn in_place<F, R>(f: F) -> R
where
F: FnOnce() -> R,
{
- use crate::executor;
+ use crate::runtime::{enter, thread_pool};
- executor::enter::exit(|| executor::thread_pool::blocking(f))
+ enter::exit(|| thread_pool::blocking(f))
}
/// Run the provided closure on a thread where blocking is acceptable.
@@ -288,7 +288,7 @@ where
///
/// ```
/// # async fn docs() {
-/// tokio::executor::blocking::run(move || {
+/// tokio::runtime::blocking::run(move || {
/// // do some compute-heavy work or call synchronous code
/// }).await;
/// # }
diff --git a/tokio/src/runtime/builder.rs b/tokio/src/runtime/builder.rs
index 1ed38c8a..92fdfa1a 100644
--- a/tokio/src/runtime/builder.rs
+++ b/tokio/src/runtime/builder.rs
@@ -1,14 +1,13 @@
-use crate::executor::blocking::{Pool, PoolWaiter};
-use crate::executor::current_thread::CurrentThread;
+#[cfg(feature = "blocking")]
+use crate::runtime::blocking::{Pool, PoolWaiter};
+#[cfg(feature = "rt-current-thread")]
+use crate::runtime::current_thread::CurrentThread;
#[cfg(feature = "rt-full")]
-use crate::executor::thread_pool;
-use crate::net::driver::Reactor;
-use crate::runtime::{Runtime, Kind};
-use crate::timer::clock::Clock;
-use crate::timer::timer::Timer;
+use crate::runtime::thread_pool;
+use crate::runtime::{io, timer, Runtime};
+use std::fmt;
use std::sync::Arc;
-use std::{fmt, io};
/// Builds Tokio Runtime with custom configuration values.
///
@@ -43,8 +42,8 @@ use std::{fmt, io};
/// }
/// ```
pub struct Builder {
- /// When `true`, use the current-thread executor.
- current_thread: bool,
+ /// The task execution model to use.
+ kind: Kind,
/// The number of worker threads.
///
@@ -64,7 +63,16 @@ pub struct Builder {
before_stop: Option<Arc<dyn Fn() + Send + Sync>>,
/// The clock to use
- clock: Clock,
+ clock: timer::Clock,
+}
+
+#[derive(Debug)]
+enum Kind {
+ Shell,
+ #[cfg(feature = "rt-current-thread")]
+ CurrentThread,
+ #[cfg(feature = "rt-full")]
+ ThreadPool,
}
impl Builder {
@@ -74,8 +82,8 @@ impl Builder {
/// Configuration methods can be chained on the return value.
pub fn new() -> Builder {
Builder {
- // Use the thread-pool executor by default
- current_thread: false,
+ // No task execution by default
+ kind: Kind::Shell,
// Default to use an equal number of threads to number of CPU cores
num_threads: crate::loom::sys::num_cpus(),
@@ -91,7 +99,7 @@ impl Builder {
before_stop: None,
// Default clock
- clock: Clock::new(),
+ clock: timer::Clock::default(),
}
}
@@ -119,12 +127,20 @@ impl Builder {
self
}
- /// Use only the current thread for the runtime.
+ /// Use only the current thread for executing tasks.
///
/// The network driver, timer, and executor will all be run on the current
/// thread during `block_on` calls.
+ #[cfg(feature = "rt-current-thread")]
pub fn current_thread(&mut self) -> &mut Self {
- self.current_thread = true;
+ self.kind = Kind::CurrentThread;
+ self
+ }
+
+ /// Use a thread-pool for executing tasks.
+ #[cfg(feature = "rt-full")]
+ pub fn thread_pool(&mut self) -> &mut Self {
+ self.kind = Kind::ThreadPool;
self
}
@@ -224,7 +240,7 @@ impl Builder {
}
/// Set the `Clock` instance that will be used by the runtime.
- pub fn clock(&mut self, clock: Clock) -> &mut Self {
+ pub fn clock(&mut self, clock: timer::Clock) -> &mut Self {
self.clock = clock;
self
}
@@ -245,20 +261,44 @@ impl Builder {
/// });
/// ```
pub fn build(&mut self) -> io::Result<Runtime> {
- if self.current_thread {
- self.build_current_thread()
- } else {
- self.build_threadpool()
+ match self.kind {
+ Kind::Shell => self.build_shell(),
+ #[cfg(feature = "rt-current-thread")]
+ Kind::CurrentThread => self.build_current_thread(),
+ #[cfg(feature = "rt-full")]
+ Kind::ThreadPool => self.build_threadpool(),
}
}
+ fn build_shell(&mut self) -> io::Result<Runtime> {
+ use crate::runtime::Kind;
+
+ // Create network driver
+ let (net, handle) = io::create()?;
+ let net_handles = vec![handle];
+
+ let (_timer, handle) = timer::create(net, self.clock.clone());
+ let timer_handles = vec![handle];
+
+ Ok(Runtime {
+ kind: Kind::Shell,
+ net_handles,
+ timer_handles,
+ #[cfg(feature = "blocking")]
+ blocking_pool: PoolWaiter::from(Pool::default()),
+ })
+ }
+
+ #[cfg(feature = "rt-current-thread")]
fn build_current_thread(&mut self) -> io::Result<Runtime> {
+ use crate::runtime::Kind;
+
// Create network driver
- let net = Reactor::new()?;
- let net_handles = vec![net.handle()];
+ let (net, handle) = io::create()?;
+ let net_handles = vec![handle];
- let timer = Timer::new_with_clock(net, self.clock.clone());
- let timer_handles = vec![timer.handle()];
+ let (timer, handle) = timer::create(net, self.clock.clone());
+ let timer_handles = vec![handle];
// And now put a single-threaded executor on top of the timer. When
// there are no futures ready to do something, it'll let the timer or
@@ -277,16 +317,10 @@ impl Builder {
})
}
- // Without rt-full, the "threadpool" variant just uses current-thread
- #[cfg(not(feature = "rt-full"))]
- fn build_threadpool(&mut self) -> io::Result<Runtime> {
- self.build_current_thread()
- }
-
#[cfg(feature = "rt-full")]
fn build_threadpool(&mut self) -> io::Result<Runtime> {
- use crate::net::driver;
- use crate::timer::{clock, timer};
+ use crate::runtime::Kind;
+ use crate::timer::clock;
use std::sync::Mutex;
let mut net_handles = Vec::new();
@@ -294,13 +328,13 @@ impl Builder {
let mut timers = Vec::new();
for _ in 0..self.num_threads {
- // Create network driver
- let net = Reactor::new()?;
- net_handles.push(net.handle());
+ // Create network driver and handle
+ let (net, handle) = io::create()?;
+ net_handles.push(handle);
// Create a new timer.
- let timer = Timer::new_with_clock(net, self.clock.clone());
- timer_handles.push(timer.handle());
+ let (timer, handle) = timer::create(net, self.clock.clone());
+ timer_handles.push(handle);
timers.push(Mutex::new(Some(timer)));
}
@@ -328,7 +362,7 @@ impl Builder {
builder
.around_worker(move |index, next| {
// Configure the network driver
- let _net = driver::set_default(&net_handles[index]);
+ let _net = io::set_default(&net_handles[index]);
// Configure the clock
clock::with_default(&clock, || {
@@ -370,7 +404,7 @@ impl Default for Builder {
impl fmt::Debug for Builder {
fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result {
fmt.debug_struct("Builder")
- .field("current_thread", &self.current_thread)
+ .field("kind", &self.kind)
.field("num_threads", &self.num_threads)
.field("thread_name", &self.thread_name)
.field("thread_stack_size", &self.thread_stack_size)
diff --git a/tokio/src/executor/current_thread/mod.rs b/tokio/src/runtime/current_thread/mod.rs
index 24fee3a8..d63b4ccf 100644
--- a/tokio/src/executor/current_thread/mod.rs
+++ b/tokio/src/runtime/current_thread/mod.rs
@@ -1,5 +1,5 @@
-use crate::executor::park::{Park, Unpark};
-use crate::executor::task::{self, JoinHandle, Schedule, Task};
+use crate::runtime::park::{Park, Unpark};
+use crate::runtime::task::{self, JoinHandle, Schedule, Task};
use std::cell::UnsafeCell;
use std::collections::VecDeque;
@@ -130,6 +130,7 @@ where
where
F: Future,
{
+ use crate::runtime;
use std::pin::Pin;
use std::task::Context;
use std::task::Poll::Ready;
@@ -137,9 +138,9 @@ where
let local = &mut self.local;
let scheduler = &*self.scheduler;
- crate::executor::global::with_current_thread(scheduler, || {
+ runtime::global::with_current_thread(scheduler, || {
let mut _enter =
- crate::executor::enter().expect("attempting to block while on a Tokio executor");
+ runtime::enter::enter().expect("attempting to block while on a Tokio executor");
let raw_waker = RawWaker::new(
scheduler as *const Scheduler as *const (),
@@ -272,7 +273,7 @@ impl Schedule for Scheduler {
}
fn schedule(&self, task: Task<Self>) {
- use crate::executor::global;
+ use crate::runtime::global;
if global::current_thread_is