summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorCarl Lerche <me@carllerche.com>2020-01-22 18:59:22 -0800
committerGitHub <noreply@github.com>2020-01-22 18:59:22 -0800
commit8cf98d694663e8397bfc337e7a4676567287bfae (patch)
tree50b214c2e29a257633494a64eed35c5b6f568eba
parentf9ea576ccae5beffeaa2f2c48c2c0d2f9449673b (diff)
Provide `select!` macro (#2152)
Provides a `select!` macro for concurrently waiting on multiple async expressions. The macro has similar goals and syntax as the one provided by the `futures` crate, but differs significantly in implementation. First, this implementation does not require special traits to be implemented on futures or streams (i.e., no `FuseFuture`). A design goal is to be able to pass a "plain" async fn result into the select! macro. Even without `FuseFuture`, this `select!` implementation is able to handle all cases the `futures::select!` macro can handle. It does this by supporting pre-poll conditions on branches and result pattern matching. For pre-conditions, each branch is able to include a condition that disables the branch if it evaluates to false. This allows the user to guard futures that have already been polled, preventing double polling. Pattern matching can be used to disable streams that complete. A second big difference is the macro is implemented almost entirely as a declarative macro. The biggest advantage to using this strategy is that the user will not need to alter the rustc recursion limit except in the most extreme cases. The resulting future also tends to be smaller in many cases.
-rw-r--r--tests-integration/Cargo.toml9
-rw-r--r--tests-integration/tests/macros_select.rs33
-rw-r--r--tokio-macros/Cargo.toml1
-rw-r--r--tokio-macros/src/lib.rs13
-rw-r--r--tokio-macros/src/select.rs43
-rw-r--r--tokio/src/future/mod.rs2
-rw-r--r--tokio/src/future/poll_fn.rs4
-rw-r--r--tokio/src/lib.rs17
-rw-r--r--tokio/src/macros/mod.rs9
-rw-r--r--tokio/src/macros/select.rs873
-rw-r--r--tokio/src/macros/support.rs6
-rw-r--r--tokio/src/util/mod.rs8
-rw-r--r--tokio/src/util/rand.rs13
-rw-r--r--tokio/tests/macros_select.rs447
14 files changed, 1468 insertions, 10 deletions
diff --git a/tests-integration/Cargo.toml b/tests-integration/Cargo.toml
index 3cca17a7..6f84afd9 100644
--- a/tests-integration/Cargo.toml
+++ b/tests-integration/Cargo.toml
@@ -6,7 +6,14 @@ edition = "2018"
publish = false
[features]
-full = ["tokio/full", "tokio-test"]
+full = [
+ "macros",
+ "rt-core",
+ "rt-threaded",
+
+ "tokio/full",
+ "tokio-test"
+]
macros = ["tokio/macros"]
rt-core = ["tokio/rt-core"]
rt-threaded = ["rt-core", "tokio/rt-threaded"]
diff --git a/tests-integration/tests/macros_select.rs b/tests-integration/tests/macros_select.rs
new file mode 100644
index 00000000..4c4fef7c
--- /dev/null
+++ b/tests-integration/tests/macros_select.rs
@@ -0,0 +1,33 @@
+#![cfg(feature = "macros")]
+
+use futures::channel::oneshot;
+use futures::executor::block_on;
+use std::thread;
+
+#[test]
+fn join_with_select() {
+ block_on(async {
+ let (tx1, mut rx1) = oneshot::channel::<i32>();
+ let (tx2, mut rx2) = oneshot::channel::<i32>();
+
+ thread::spawn(move || {
+ tx1.send(123).unwrap();
+ tx2.send(456).unwrap();
+ });
+
+ let mut a = None;
+ let mut b = None;
+
+ while a.is_none() || b.is_none() {
+ tokio::select! {
+ v1 = (&mut rx1), if a.is_none() => a = Some(v1.unwrap()),
+ v2 = (&mut rx2), if b.is_none() => b = Some(v2.unwrap()),
+ }
+ }
+
+ let (a, b) = (a.unwrap(), b.unwrap());
+
+ assert_eq!(a, 123);
+ assert_eq!(b, 456);
+ });
+}
diff --git a/tokio-macros/Cargo.toml b/tokio-macros/Cargo.toml
index d69d48d6..3cc3a3e4 100644
--- a/tokio-macros/Cargo.toml
+++ b/tokio-macros/Cargo.toml
@@ -25,6 +25,7 @@ proc-macro = true
[features]
[dependencies]
+proc-macro2 = "1.0.7"
quote = "1"
syn = { version = "1.0.3", features = ["full"] }
diff --git a/tokio-macros/src/lib.rs b/tokio-macros/src/lib.rs
index 09b8b093..17554470 100644
--- a/tokio-macros/src/lib.rs
+++ b/tokio-macros/src/lib.rs
@@ -14,10 +14,11 @@
//! Macros for use with Tokio
-mod entry;
-
extern crate proc_macro;
+mod entry;
+mod select;
+
use proc_macro::TokenStream;
/// Marks async function to be executed by selected runtime.
@@ -198,3 +199,11 @@ pub fn test(args: TokenStream, item: TokenStream) -> TokenStream {
pub fn test_basic(args: TokenStream, item: TokenStream) -> TokenStream {
entry::test(args, item, false)
}
+
+/// Implementation detail of the `select!` macro. This macro is **not** intended
+/// to be used as part of the public API and is permitted to change.
+#[proc_macro]
+#[doc(hidden)]
+pub fn select_priv_declare_output_enum(input: TokenStream) -> TokenStream {
+ select::declare_output_enum(input)
+}
diff --git a/tokio-macros/src/select.rs b/tokio-macros/src/select.rs
new file mode 100644
index 00000000..ddb2e6a0
--- /dev/null
+++ b/tokio-macros/src/select.rs
@@ -0,0 +1,43 @@
+use proc_macro::{TokenStream, TokenTree};
+use proc_macro2::Span;
+use quote::quote;
+use syn::Ident;
+
+pub(crate) fn declare_output_enum(input: TokenStream) -> TokenStream {
+ // passed in is: `(_ _ _)` with one `_` per branch
+ let branches = match input.into_iter().next() {
+ Some(TokenTree::Group(group)) => group.stream().into_iter().count(),
+ _ => panic!("unexpected macro input"),
+ };
+
+ let variants = (0..branches)
+ .map(|num| Ident::new(&format!("_{}", num), Span::call_site()))
+ .collect::<Vec<_>>();
+
+ // Use a bitfield to track which futures completed
+ let mask = Ident::new(
+ if branches <= 8 {
+ "u8"
+ } else if branches <= 16 {
+ "u16"
+ } else if branches <= 32 {
+ "u32"
+ } else if branches <= 64 {
+ "u64"
+ } else {
+ panic!("up to 64 branches supported");
+ },
+ Span::call_site(),
+ );
+
+ TokenStream::from(quote! {
+ pub(super) enum Out<#( #variants ),*> {
+ #( #variants(#variants), )*
+ // Include a `Disabled` variant signifying that all select branches
+ // failed to resolve.
+ Disabled,
+ }
+
+ pub(super) type Mask = #mask;
+ })
+}
diff --git a/tokio/src/future/mod.rs b/tokio/src/future/mod.rs
index 9a155bf7..c5225600 100644
--- a/tokio/src/future/mod.rs
+++ b/tokio/src/future/mod.rs
@@ -6,7 +6,7 @@ mod maybe_done;
pub(crate) use maybe_done::{maybe_done, MaybeDone};
mod poll_fn;
-pub(crate) use poll_fn::poll_fn;
+pub use poll_fn::poll_fn;
mod ready;
pub(crate) use ready::{ok, Ready};
diff --git a/tokio/src/future/poll_fn.rs b/tokio/src/future/poll_fn.rs
index ce2a5524..9b3d1370 100644
--- a/tokio/src/future/poll_fn.rs
+++ b/tokio/src/future/poll_fn.rs
@@ -6,14 +6,14 @@ use std::pin::Pin;
use std::task::{Context, Poll};
/// Future for the [`poll_fn`] function.
-pub(crate) struct PollFn<F> {
+pub struct PollFn<F> {
f: F,
}
impl<F> Unpin for PollFn<F> {}
/// Creates a new future wrapping around a function returning [`Poll`].
-pub(crate) fn poll_fn<T, F>(f: F) -> PollFn<F>
+pub fn poll_fn<T, F>(f: F) -> PollFn<F>
where
F: FnMut(&mut Context<'_>) -> Poll<T>,
{
diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs
index 42a34131..7989d5b3 100644
--- a/tokio/src/lib.rs
+++ b/tokio/src/lib.rs
@@ -282,15 +282,20 @@
//! }
//! ```
-// macros used internally
+// Includes re-exports used by macros.
+//
+// This module is not intended to be part of the public API. In general, any
+// `doc(hidden)` code is not part of Tokio's public and stable API.
#[macro_use]
-mod macros;
+#[doc(hidden)]
+pub mod macros;
cfg_fs! {
pub mod fs;
}
-mod future;
+#[doc(hidden)]
+pub mod future;
pub mod io;
pub mod net;
@@ -333,6 +338,12 @@ cfg_time! {
mod util;
cfg_macros! {
+ /// Implementation detail of the `select!` macro. This macro is **not**
+ /// intended to be used as part of the public API and is permitted to
+ /// change.
+ #[doc(hidden)]
+ pub use tokio_macros::select_priv_declare_output_enum;
+
doc_rt_core! {
cfg_rt_threaded! {
#[cfg(not(test))] // Work around for rust-lang/rust#62127
diff --git a/tokio/src/macros/mod.rs b/tokio/src/macros/mod.rs
index 9136e594..708dd683 100644
--- a/tokio/src/macros/mod.rs
+++ b/tokio/src/macros/mod.rs
@@ -14,4 +14,13 @@ mod loom;
mod ready;
#[macro_use]
+mod select;
+
+#[macro_use]
mod thread_local;
+
+cfg_macros! {
+ // Includes re-exports needed to implement macros
+ #[doc(hidden)]
+ pub mod support;
+}
diff --git a/tokio/src/macros/select.rs b/tokio/src/macros/select.rs
new file mode 100644
index 00000000..00c2c6b5
--- /dev/null
+++ b/tokio/src/macros/select.rs
@@ -0,0 +1,873 @@
+/// Wait on multiple concurrent branches, returning when the **first** branch
+/// completes, cancelling the remaining branches.
+///
+/// The `select` macro accepts one or more branches with the following pattern:
+///
+/// ```text
+/// <pattern> = <async expression> (, if <precondition>)? => <handler>,
+/// ```
+///
+/// Additionally, the `select!` macro may include a single, optional `else`
+/// branch, which evaluates if none of the other branches match their patterns:
+///
+/// ```text
+/// else <expression>
+/// ```
+///
+/// The macro aggregates all `<async expression>` expressions and runs them
+/// concurrently on the **current** task. Once the **first** expression
+/// completes with a value that matches its `<pattern>`, the `select!` macro
+/// returns the result of evaluating the completed branch's `<handler>`
+/// expression.
+///
+/// Additionally, each branch may include an optional `if` precondition. This
+/// precondition is evaluated **before** the <async expression>. If the
+/// precondition returns `false`, the branch is entirely disabled. This
+/// capability is useful when using `select!` within a loop.
+///
+/// The complete lifecycle of a `select!` expression is as follows:
+///
+/// 1. Evaluate all provded `<precondition>` expressions. If the precondition
+/// returns `false`, disable the branch for the remainder of the current call
+/// to `select!`. Re-entering `select!` due to a loop clears the "disabled"
+/// state.
+/// 2. Aggregate the `<async expression>`s from each branch, including the
+/// disabled ones. If the branch is disabled, `<async expression>` is still
+/// evaluated, but the resulting future is not polled.
+/// 3. Concurrently await on the results for all remaining `<async expression>`s.
+/// 4. Once an `<async expression>` returns a value, attempt to apply the value
+/// to the provided `<pattern>`, if the pattern matches, evaluate `<handler>`
+/// and return. If the pattern **does not** match, disable the current branch
+/// and for the remainder of the current call to `select!. Continue from step 3.
+/// 5. If **all** branches are disabled, evaluate the `else` expression. If none
+/// is provided, panic.
+///
+/// # Notes
+///
+/// ### Runtime characteristics
+///
+/// By running all async expressions on the current task, the expressions are
+/// able to run **concurrently** but not in **parallel**. This means all
+/// expressions are run on the same thread and if one branch blocks the thread,
+/// all other expressions will be unable to continue. If parallelism is
+/// required, spawn each async expression using [`tokio::spawn`] and pass the
+/// join handle to `select!`.
+///
+/// [`tokio::spawn`]: crate::spawn
+///
+/// ### Avoid racy `if` preconditions
+///
+/// Given that `if` preconditions are used to disable `select!` branches, some
+/// caution must be used to avoid missing values.
+///
+/// For example, here is **incorrect** usage of `delay` with `if`. The objective
+/// is to repeatedly run an asynchronous task for up to 50 milliseconds.
+/// However, there is a potential for the `delay` completion to be missed.
+///
+/// ```no_run
+/// use tokio::time::{self, Duration};
+///
+/// async fn some_async_work() {
+/// // do work
+/// }
+///
+/// #[tokio::main]
+/// async fn main() {
+/// let mut delay = time::delay_for(Duration::from_millis(50));
+///
+/// while !delay.is_elapsed() {
+/// tokio::select! {
+/// _ = &mut delay, if !delay.is_elapsed() => {
+/// println!("operation timed out");
+/// }
+/// _ = some_async_work() => {
+/// println!("operation completed");
+/// }
+/// }
+/// }
+/// }
+/// ```
+///
+/// In the above example, `delay.is_elapsed()` may return `true` even if
+/// `delay.poll()` never returned `Ready`. This opens up a potential race
+/// condition where `delay` expires between the `while !delay.is_elapsed()`
+/// check and the call to `select!` resulting in the `some_async_work()` call to
+/// run uninterrupted despite the delay having elapsed.
+///
+/// One way to write the above example without the race would be:
+///
+/// ```
+/// use tokio::time::{self, Duration};
+///
+/// async fn some_async_work() {
+/// # time::delay_for(Duration::from_millis(10)).await;
+/// // do work
+/// }
+///
+/// #[tokio::main]
+/// async fn main() {
+/// let mut delay = time::delay_for(Duration::from_millis(50));
+///
+/// loop {
+/// tokio::select! {
+/// _ = &mut delay => {
+/// println!("operation timed out");
+/// break;
+/// }
+/// _ = some_async_work() => {
+/// println!("operation completed");
+/// }
+/// }
+/// }
+/// }
+/// ```
+///
+/// ### Fairness
+///
+/// `select!` randomly picks a branch to check first. This provides some level
+/// of fairness when calling `select!` in a loop with branches that are always
+/// ready.
+///
+/// # Panics
+///
+/// `select!` panics if all branches are disabled **and** there is no provided
+/// `else` branch. A branch is disabled when the provided `if` precondition
+/// returns `false` **or** when the pattern does not match the result of `<async
+/// expression>.
+///
+/// # Examples
+///
+/// Basic select with two branches.
+///
+/// ```
+/// async fn do_stuff_async() {
+/// // async work
+/// }
+///
+/// async fn more_async_work() {
+/// // more here
+/// }
+///
+/// #[tokio::main]
+/// async fn main() {
+/// tokio::select! {
+/// _ = do_stuff_async() => {
+/// println!("do_stuff_async() completed first")
+/// }
+/// _ = more_async_work() => {
+/// println!("more_async_work() completed first")
+/// }
+/// };
+/// }
+///
+/// ```
+///
+/// Basic stream selecting.
+///
+/// ```
+/// use tokio::stream::{self, StreamExt};
+///
+/// #[tokio::main]
+/// async fn main() {
+/// let mut stream1 = stream::iter(vec![1, 2, 3]);
+/// let mut stream2 = stream::iter(vec![4, 5, 6]);
+///
+/// let next = tokio::select! {
+/// v = stream1.next() => v.unwrap(),
+/// v = stream2.next() => v.unwrap(),
+/// };
+///
+/// assert!(next == 1 || next == 4);
+/// }
+/// ```
+///
+/// Collect the contents of two streams. In this example, we rely on pattern
+/// matching and the fact that `stream::iter` is "fused", i.e. once the stream
+/// is complete, all calls to `next()` return `None`.
+///
+/// ```
+/// use tokio::stream::{self, StreamExt};
+///
+/// #[tokio::main]
+/// async fn main() {
+/// let mut stream1 = stream::iter(vec![1, 2, 3]);
+/// let mut stream2 = stream::iter(vec![4, 5, 6]);
+///
+/// let mut values = vec![];
+///
+/// loop {
+/// tokio::select! {
+/// Some(v) = stream1.next() => values.push(v),
+/// Some(v) = stream2.next() => values.push(v),
+/// else => break,
+/// }
+/// }
+///
+/// values.sort();
+/// assert_eq!(&[1, 2, 3, 4, 5, 6], &values[..]);
+/// }
+/// ```
+///
+/// Using the same future in multiple select! expressions can be done by passing
+/// a reference to the future. Doing so requires the future to be [`Unpin`]. A
+/// future can be made [`Unpin`] by either using [`Box::pin`] or stack pinning.
+///
+/// [`Unpin`]: std::marker::Unpin
+/// [`Box::pin`]: std::boxed::Box::pin
+///
+/// Here, a stream is consumed for at most 1 second.
+///
+/// ```
+/// use tokio::stream::{self, StreamExt};
+/// use tokio::time::{self, Duration};
+///
+/// #[tokio::main]
+/// async fn main() {
+/// let mut stream = stream::iter(vec![1, 2, 3]);
+/// let mut delay = time::delay_for(Duration::from_secs(1));
+///
+/// loop {
+/// tokio::select! {
+/// maybe_v = stream.next() => {
+/// if let Some(v) = maybe_v {
+/// println!("got = {}", v);
+/// } else {
+/// break;
+/// }
+/// }
+/// _ = &mut delay => {
+/// println!("timeout");
+/// break;
+/// }
+/// }
+/// }
+/// }
+/// ```
+///
+/// Joining two values using `select!`.
+///
+/// ```
+/// use tokio::sync::oneshot;
+///
+/// #[tokio::main]
+/// async fn main() {
+/// let (tx1, mut rx1) = oneshot::channel();
+/// let (tx2, mut rx2) = oneshot::channel();
+///
+/// tokio::spawn(async move {
+/// tx1.send("first").unwrap();
+/// });
+///
+/// tokio::spawn(async move {
+/// tx2.send("second").unwrap();
+/// });
+///
+/// let mut a = None;
+/// let mut b = None;
+///
+/// while a.is_none() || b.is_none() {
+/// tokio::select! {
+/// v1 = (&mut rx1), if a.is_none() => a = Some(v1.unwrap()),
+/// v2 = (&mut rx2), if b.is_none() => b = Some(v2.unwrap()),
+/// }
+/// }
+///
+/// let res = (a.unwrap(), b.unwrap());
+///
+/// assert_eq!(res.0, "first");
+/// assert_eq!(res.1, "second");
+/// }
+/// ```
+#[macro_export]
+macro_rules! select {
+ // Uses a declarative macro to do **most** of the work. While it is possible
+ // to implement fully with a declarative macro, a procedural macro is used
+ // to enable improved error messages.
+ //
+ // The macro is structured as a tt-muncher. All branches are processed and
+ // normalized. Once the input is normalized, it is passed to the top-most
+ // rule. When entering the macro, `@{ }` is inserted at the front. This is
+ // used to collect the normalized input.
+ //
+ // The macro only recurses once per branch. This allows using `select!`
+ // without requiring the user to increase the recursion limit.
+
+ // All input is normalized, now transform.
+ (@ {
+ // One `_` for each branch in the `select!` macro. Passing this to
+ // `count!` converts $skip to an integer.
+ ( $($count:tt)* )
+
+ // Normalized select branches. `( $skip )` is a set of `_` characters.
+ // There is one `_` for each select branch **before** this one. Given
+ // that all input futures are stored in a tuple, $skip is useful for
+ // generating a pattern to reference the future for the current branch.
+ // $skip is also used as an argument to `count!`, returning the index of
+ // the current select branch.
+ $( ( $($skip:tt)* ) $bind:pat = $fut:expr, if $c:expr => $handle:expr, )+
+
+ // Fallback expression used when all select branches have been disabled.
+ ; $else:expr
+
+ }) => {{
+ // Enter a context where stable "function-like" proc macros can be used.
+ //
+ // This module is defined within a scope and should not leak out of this
+ // macro.
+ mod util {
+ // Generate an enum with one variant per select branch
+ $crate::select_priv_declare_output_enum!( ( $($count)* ) );
+ }
+
+ // `tokio::macros::support` is a public, but doc(hidden) module
+ // including a re-export of all types needed by this macro.
+ use $crate::macros::support::Future;
+ use $crate::macros::support::Pin;
+ use $crate::macros::support::Poll::{Ready, Pending};
+
+ const BRANCHES: u32 = $crate::count!( $($count)* );
+
+ let mut disabled: util::Mask = Default::default();
+
+ // First, invoke all the pre-conditions. For any that return true,
+ // set the appropriate bit in `disabled`.
+ $(
+ if !$c {
+ let mask = 1 << $crate::count!( $($skip)* );
+ disabled |= mask;
+ }
+ )*
+
+ // Create a scope to separate polling from handling the output. This
+ // adds borrow checker flexibility when using the macro.
+ let mut output = {
+ // Safety: Nothing must be moved out of `futures`. This is to
+ // satisfy the requirement of `Pin::new_unchecked` called below.
+ let mut futures = ( $( $fut , )+ );
+
+ $crate::macros::support::poll_fn(|cx| {
+ // Track if any branch returns pending. If no branch completes
+ // **or** returns pending, this implies that all branches are
+ // disabled.
+ let mut is_pending = false;
+
+ // Randomly generate a starting point. This makes `select!` a
+ // bit more fair and avoids always polling the first future.
+ let start = $crate::macros::support::thread_rng_n(BRANCHES);
+
+ for i in 0..BRANCHES {
+ let branch = (start + i) % BRANCHES;
+
+ match branch {
+ $(
+ $crate::count!( $($skip)* ) => {
+ // First, if the future has previously been
+ // disabled, do not poll it again. This is done
+ // by checking the associated bit in the
+ // `disabled` bit field.
+ let mask = 1 << branch;
+
+ if disabled & mask == mask {
+ // The future has been disabled.
+ continue;
+ }
+
+ // Extract the future for this branch from the
+ // tuple
+ let ( $($skip,)* fut, .. ) = &mut futures;
+
+ // Safety: future is stored on the stack above
+ // and never moved.
+ let mut fut = unsafe { Pin::new_unchecked(fut) };
+
+ // Try polling it
+ let out = match fut.poll(cx) {
+ Ready(out) => out,
+ Pending => {
+ // Track that at least one future is
+ // still pending and continue polling.
+ is_pending = true;
+ continue;
+ }
+ };
+
+ // Disable the future from future polling.
+ disabled |= mask;
+
+ // The future returned a value, check if matches
+ // the specified pattern.
+ #[allow(unused_variables)]
+ match &out {
+ $bind => {}
+ _ => continue,
+ }
+
+ // The select is complete, return the value
+ return Ready($crate::select_variant!(util::Out, ($($skip)*))(out));
+ }
+ )*
+ _ => unreachable!("reaching this means there probably is an off by one bug"),
+ }
+ }
+
+ if is_pending {
+ Pending
+ } else {
+ // All branches have been disabled.
+ Ready(util::Out::Disabled)
+ }
+ }).await
+ };
+
+ match output {
+ $(
+ $crate::select_variant!(util::Out, ($($skip)*) ($bind)) => $handle,
+ )*
+ util::Out::Disabled => $else,
+ _ => unreachable!("failed to match bind"),
+ }
+ }};
+
+ // ==== Normalize =====
+
+ // These rules match a single `select!` branch and normalize it for
+ // processing by the first rule.
+
+ (@ { $($t:tt)* } ) => {
+ // No `else` branch
+ $crate::select!(@{ $($t)*; unreachable!() })
+ };
+ (@ { $($t:tt)* } else => $else:expr $(,)?) => {
+ $crate::select!(@{ $($t)*; $else })
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr, if $c:expr => $h:block, $($r:tt)* ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if $c => $h, } $($r)*)
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr => $h:block, $($r:tt)* ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if true => $h, } $($r)*)
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr, if $c:expr => $h:block $($r:tt)* ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if $c => $h, } $($r)*)
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr => $h:block $($r:tt)* ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if true => $h, } $($r)*)
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr, if $c:expr => $h:expr ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if $c => $h, })
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr => $h:expr ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if true => $h, })
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr, if $c:expr => $h:expr, $($r:tt)* ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if $c => $h, } $($r)*)
+ };
+ (@ { ( $($s:tt)* ) $($t:tt)* } $p:pat = $f:expr => $h:expr, $($r:tt)* ) => {
+ $crate::select!(@{ ($($s)* _) $($t)* ($($s)*) $p = $f, if true => $h, } $($r)*)
+ };
+
+ // ===== Entry point =====
+
+ ( $p:pat = $($t:tt)* ) => {
+ $crate::select!(@{ () } $p = $($t)*)
+ };
+ () => {
+ compile_error!("select! requires at least one branch.")
+ };
+}
+
+// And here... we manually list out matches for up to 64 branches... I'm not
+// happy about it either, but this is how we manage to use a declarative macro!
+
+#[macro_export]
+#[doc(hidden)]
+macro_rules! count {
+ () => {
+ 0
+ };
+ (_) => {
+ 1
+ };
+ (_ _) => {
+ 2
+ };
+ (_ _ _) => {
+ 3
+ };
+ (_ _ _ _) => {
+ 4
+ };
+ (_ _ _ _ _) => {
+ 5
+ };
+ (_ _ _ _ _ _) => {
+ 6
+ };
+ (_ _ _ _ _ _ _) => {
+ 7
+ };
+ (_ _ _ _ _ _ _ _) => {
+ 8
+ };
+ (_ _ _ _ _ _ _ _ _) => {
+ 9
+ };
+ (_ _ _ _ _ _ _ _ _ _) => {
+ 10
+ };
+ (_ _ _ _ _ _ _ _ _ _ _) => {
+ 11
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _) => {
+ 12
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 13
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 14
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 15
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 16
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 17
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 18
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 19
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 20
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 21
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 22
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 23
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 24
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 25
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 26
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 27
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 28
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 29
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 30
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 31
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 32
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 33
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 34
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 35
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 36
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 37
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 38
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 39
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 40
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 41
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 42
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 43
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 44
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 45
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 46
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 47
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 48
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 49
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 50
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 51
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 52
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 53
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 54
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 55
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 56
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 57
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 58
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 59
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 60
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 61
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 62
+ };
+ (_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _) => {
+ 63
+ };
+}
+
+#[macro_export]
+#[doc(hidden)]
+macro_rules! select_variant {
+ ($($p:ident)::*, () $($t:tt)*) => {
+ $($p)::*::_0 $($t)*
+ }