mirror of
https://github.com/tokio-rs/tokio.git
synced 2025-09-28 12:10:37 +00:00
doc: document cancellation safety (#3900)
This patch documents cancellation safety. It also moves the "Avoid racy if preconditions" section in the select! documentation since otherwise the first code block on the page shows how not to use it, which seems counterintuitive.
This commit is contained in:
parent
57c90c9750
commit
b521cc2689
@ -105,8 +105,8 @@ cfg_io_util! {
|
||||
/// async fn read(&mut self, buf: &mut [u8]) -> io::Result<usize>;
|
||||
/// ```
|
||||
///
|
||||
/// This function does not provide any guarantees about whether it
|
||||
/// completes immediately or asynchronously
|
||||
/// This method does not provide any guarantees about whether it
|
||||
/// completes immediately or asynchronously.
|
||||
///
|
||||
/// # Return
|
||||
///
|
||||
@ -138,6 +138,12 @@ cfg_io_util! {
|
||||
/// variant will be returned. If an error is returned then it must be
|
||||
/// guaranteed that no bytes were read.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If you use it as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that no data was read.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// [`File`][crate::fs::File]s implement `Read`:
|
||||
@ -177,8 +183,8 @@ cfg_io_util! {
|
||||
/// Usually, only a single `read` syscall is issued, even if there is
|
||||
/// more space in the supplied buffer.
|
||||
///
|
||||
/// This function does not provide any guarantees about whether it
|
||||
/// completes immediately or asynchronously
|
||||
/// This method does not provide any guarantees about whether it
|
||||
/// completes immediately or asynchronously.
|
||||
///
|
||||
/// # Return
|
||||
///
|
||||
@ -197,6 +203,12 @@ cfg_io_util! {
|
||||
/// variant will be returned. If an error is returned then it must be
|
||||
/// guaranteed that no bytes were read.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If you use it as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that no data was read.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// [`File`] implements `Read` and [`BytesMut`] implements [`BufMut`]:
|
||||
@ -261,6 +273,13 @@ cfg_io_util! {
|
||||
/// it has read, but it will never read more than would be necessary to
|
||||
/// completely fill the buffer.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is not cancellation safe. If the method is used as the
|
||||
/// event in a [`tokio::select!`](crate::select) statement and some
|
||||
/// other branch completes first, then some data may already have been
|
||||
/// read into `buf`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// [`File`][crate::fs::File]s implement `Read`:
|
||||
|
@ -97,6 +97,13 @@ cfg_io_util! {
|
||||
/// It is **not** considered an error if the entire buffer could not be
|
||||
/// written to this writer.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancellation safe in the sense that if it is used as
|
||||
/// the event in a [`tokio::select!`](crate::select) statement and some
|
||||
/// other branch completes first, then it is guaranteed that no data was
|
||||
/// written to this `AsyncWrite`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -129,6 +136,13 @@ cfg_io_util! {
|
||||
///
|
||||
/// See [`AsyncWrite::poll_write_vectored`] for more details.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancellation safe in the sense that if it is used as
|
||||
/// the event in a [`tokio::select!`](crate::select) statement and some
|
||||
/// other branch completes first, then it is guaranteed that no data was
|
||||
/// written to this `AsyncWrite`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -195,6 +209,13 @@ cfg_io_util! {
|
||||
/// It is **not** considered an error if the entire buffer could not be
|
||||
/// written to this writer.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancellation safe in the sense that if it is used as
|
||||
/// the event in a [`tokio::select!`](crate::select) statement and some
|
||||
/// other branch completes first, then it is guaranteed that no data was
|
||||
/// written to this `AsyncWrite`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// [`File`] implements [`AsyncWrite`] and [`Cursor`]`<&[u8]>` implements [`Buf`]:
|
||||
@ -243,6 +264,7 @@ cfg_io_util! {
|
||||
/// while buf.has_remaining() {
|
||||
/// self.write_buf(&mut buf).await?;
|
||||
/// }
|
||||
/// Ok(())
|
||||
/// }
|
||||
/// ```
|
||||
///
|
||||
@ -254,6 +276,15 @@ cfg_io_util! {
|
||||
/// The buffer is advanced after each chunk is successfully written. After failure,
|
||||
/// `src.chunk()` will return the chunk that failed to write.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// If `write_all_buf` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then the data in the provided buffer may have been
|
||||
/// partially written. However, it is guaranteed that the provided
|
||||
/// buffer has been [advanced] by the amount of bytes that have been
|
||||
/// partially written.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// [`File`] implements [`AsyncWrite`] and [`Cursor`]`<&[u8]>` implements [`Buf`]:
|
||||
@ -261,6 +292,7 @@ cfg_io_util! {
|
||||
/// [`File`]: crate::fs::File
|
||||
/// [`Buf`]: bytes::Buf
|
||||
/// [`Cursor`]: std::io::Cursor
|
||||
/// [advanced]: bytes::Buf::advance
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio::io::{self, AsyncWriteExt};
|
||||
@ -300,6 +332,14 @@ cfg_io_util! {
|
||||
/// has been successfully written or such an error occurs. The first
|
||||
/// error generated from this method will be returned.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is not cancellation safe. If it is used as the event
|
||||
/// in a [`tokio::select!`](crate::select) statement and some other
|
||||
/// branch completes first, then the provided buffer may have been
|
||||
/// partially written, but future calls to `write_all` will start over
|
||||
/// from the beginning of the buffer.
|
||||
///
|
||||
/// # Errors
|
||||
///
|
||||
/// This function will return the first error that [`write`] returns.
|
||||
|
@ -23,10 +23,10 @@
|
||||
/// 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.
|
||||
/// Additionally, each branch may include an optional `if` precondition. If the
|
||||
/// precondition returns `false`, then the branch is disabled. The provided
|
||||
/// `<async expression>` is still evaluated but the resulting future is never
|
||||
/// polled. This capability is useful when using `select!` within a loop.
|
||||
///
|
||||
/// The complete lifecycle of a `select!` expression is as follows:
|
||||
///
|
||||
@ -42,12 +42,10 @@
|
||||
/// 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.
|
||||
/// 5. If **all** branches are disabled, evaluate the `else` expression. If no
|
||||
/// else branch is provided, panic.
|
||||
///
|
||||
/// # Notes
|
||||
///
|
||||
/// ### Runtime characteristics
|
||||
/// # 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
|
||||
@ -58,76 +56,7 @@
|
||||
///
|
||||
/// [`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 `sleep` with `if`. The objective
|
||||
/// is to repeatedly run an asynchronous task for up to 50 milliseconds.
|
||||
/// However, there is a potential for the `sleep` completion to be missed.
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio::time::{self, Duration};
|
||||
///
|
||||
/// async fn some_async_work() {
|
||||
/// // do work
|
||||
/// }
|
||||
///
|
||||
/// #[tokio::main]
|
||||
/// async fn main() {
|
||||
/// let sleep = time::sleep(Duration::from_millis(50));
|
||||
/// tokio::pin!(sleep);
|
||||
///
|
||||
/// while !sleep.is_elapsed() {
|
||||
/// tokio::select! {
|
||||
/// _ = &mut sleep, if !sleep.is_elapsed() => {
|
||||
/// println!("operation timed out");
|
||||
/// }
|
||||
/// _ = some_async_work() => {
|
||||
/// println!("operation completed");
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// ```
|
||||
///
|
||||
/// In the above example, `sleep.is_elapsed()` may return `true` even if
|
||||
/// `sleep.poll()` never returned `Ready`. This opens up a potential race
|
||||
/// condition where `sleep` expires between the `while !sleep.is_elapsed()`
|
||||
/// check and the call to `select!` resulting in the `some_async_work()` call to
|
||||
/// run uninterrupted despite the sleep 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::sleep(Duration::from_millis(10)).await;
|
||||
/// // do work
|
||||
/// }
|
||||
///
|
||||
/// #[tokio::main]
|
||||
/// async fn main() {
|
||||
/// let sleep = time::sleep(Duration::from_millis(50));
|
||||
/// tokio::pin!(sleep);
|
||||
///
|
||||
/// loop {
|
||||
/// tokio::select! {
|
||||
/// _ = &mut sleep => {
|
||||
/// println!("operation timed out");
|
||||
/// break;
|
||||
/// }
|
||||
/// _ = some_async_work() => {
|
||||
/// println!("operation completed");
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// ```
|
||||
///
|
||||
/// ### Fairness
|
||||
/// # Fairness
|
||||
///
|
||||
/// By default, `select!` randomly picks a branch to check first. This provides
|
||||
/// some level of fairness when calling `select!` in a loop with branches that
|
||||
@ -151,10 +80,60 @@
|
||||
///
|
||||
/// # 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>`.
|
||||
/// The `select!` macro 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>`.
|
||||
///
|
||||
/// # Cancellation safety
|
||||
///
|
||||
/// When using `select!` in a loop to receive messages from multiple sources,
|
||||
/// you should make sure that the receive call is cancellation safe to avoid
|
||||
/// losing messages. This section goes through various common methods and
|
||||
/// describes whether they are cancel safe. The lists in this section are not
|
||||
/// exhaustive.
|
||||
///
|
||||
/// The following methods are cancellation safe:
|
||||
///
|
||||
/// * [`tokio::sync::mpsc::Receiver::recv`](crate::sync::mpsc::Receiver::recv)
|
||||
/// * [`tokio::sync::mpsc::UnboundedReceiver::recv`](crate::sync::mpsc::UnboundedReceiver::recv)
|
||||
/// * [`tokio::sync::broadcast::Receiver::recv`](crate::sync::broadcast::Receiver::recv)
|
||||
/// * [`tokio::sync::watch::Receiver::changed`](crate::sync::watch::Receiver::changed)
|
||||
/// * [`tokio::net::TcpListener::accept`](crate::net::TcpListener::accept)
|
||||
/// * [`tokio::net::UnixListener::accept`](crate::net::UnixListener::accept)
|
||||
/// * [`tokio::io::AsyncReadExt::read`](crate::io::AsyncReadExt::read) on any `AsyncRead`
|
||||
/// * [`tokio::io::AsyncReadExt::read_buf`](crate::io::AsyncReadExt::read_buf) on any `AsyncRead`
|
||||
/// * [`tokio::io::AsyncWriteExt::write`](crate::io::AsyncWriteExt::write) on any `AsyncWrite`
|
||||
/// * [`tokio::io::AsyncWriteExt::write_buf`](crate::io::AsyncWriteExt::write_buf) on any `AsyncWrite`
|
||||
/// * [`tokio_stream::StreamExt::next`](https://docs.rs/tokio-stream/0.1/tokio_stream/trait.StreamExt.html#method.next) on any `Stream`
|
||||
/// * [`futures::stream::StreamExt::next`](https://docs.rs/futures/0.3/futures/stream/trait.StreamExt.html#method.next) on any `Stream`
|
||||
///
|
||||
/// The following methods are not cancellation safe and can lead to loss of data:
|
||||
///
|
||||
/// * [`tokio::io::AsyncReadExt::read_exact`](crate::io::AsyncReadExt::read_exact)
|
||||
/// * [`tokio::io::AsyncReadExt::read_to_end`](crate::io::AsyncReadExt::read_to_end)
|
||||
/// * [`tokio::io::AsyncReadExt::read_to_string`](crate::io::AsyncReadExt::read_to_string)
|
||||
/// * [`tokio::io::AsyncWriteExt::write_all`](crate::io::AsyncWriteExt::write_all)
|
||||
///
|
||||
/// The following methods are not cancellation safe because they use a queue for
|
||||
/// fairness and cancellation makes you lose your place in the queue:
|
||||
///
|
||||
/// * [`tokio::sync::Mutex::lock`](crate::sync::Mutex::lock)
|
||||
/// * [`tokio::sync::RwLock::read`](crate::sync::RwLock::read)
|
||||
/// * [`tokio::sync::RwLock::write`](crate::sync::RwLock::write)
|
||||
/// * [`tokio::sync::Semaphore::acquire`](crate::sync::Semaphore::acquire)
|
||||
/// * [`tokio::sync::Notify::notified`](crate::sync::Notify::notified)
|
||||
///
|
||||
/// To determine whether your own methods are cancellation safe, look for the
|
||||
/// location of uses of `.await`. This is because when an asynchronous method is
|
||||
/// cancelled, that always happens at an `.await`. If your function behaves
|
||||
/// correctly even if it is restarted while waiting at an `.await`, then it is
|
||||
/// cancellation safe.
|
||||
///
|
||||
/// Be aware that cancelling something that is not cancellation safe is not
|
||||
/// necessarily wrong. For example, if you are cancelling a task because the
|
||||
/// application is shutting down, then you probably don't care that partially
|
||||
/// read data is lost.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
@ -338,6 +317,77 @@
|
||||
/// }
|
||||
/// }
|
||||
/// ```
|
||||
///
|
||||
/// ## 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 `sleep` with `if`. The objective
|
||||
/// is to repeatedly run an asynchronous task for up to 50 milliseconds.
|
||||
/// However, there is a potential for the `sleep` completion to be missed.
|
||||
///
|
||||
/// ```no_run,should_panic
|
||||
/// use tokio::time::{self, Duration};
|
||||
///
|
||||
/// async fn some_async_work() {
|
||||
/// // do work
|
||||
/// }
|
||||
///
|
||||
/// #[tokio::main]
|
||||
/// async fn main() {
|
||||
/// let sleep = time::sleep(Duration::from_millis(50));
|
||||
/// tokio::pin!(sleep);
|
||||
///
|
||||
/// while !sleep.is_elapsed() {
|
||||
/// tokio::select! {
|
||||
/// _ = &mut sleep, if !sleep.is_elapsed() => {
|
||||
/// println!("operation timed out");
|
||||
/// }
|
||||
/// _ = some_async_work() => {
|
||||
/// println!("operation completed");
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
///
|
||||
/// panic!("This example shows how not to do it!");
|
||||
/// }
|
||||
/// ```
|
||||
///
|
||||
/// In the above example, `sleep.is_elapsed()` may return `true` even if
|
||||
/// `sleep.poll()` never returned `Ready`. This opens up a potential race
|
||||
/// condition where `sleep` expires between the `while !sleep.is_elapsed()`
|
||||
/// check and the call to `select!` resulting in the `some_async_work()` call to
|
||||
/// run uninterrupted despite the sleep 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::sleep(Duration::from_millis(10)).await;
|
||||
/// // do work
|
||||
/// }
|
||||
///
|
||||
/// #[tokio::main]
|
||||
/// async fn main() {
|
||||
/// let sleep = time::sleep(Duration::from_millis(50));
|
||||
/// tokio::pin!(sleep);
|
||||
///
|
||||
/// loop {
|
||||
/// tokio::select! {
|
||||
/// _ = &mut sleep => {
|
||||
/// println!("operation timed out");
|
||||
/// break;
|
||||
/// }
|
||||
/// _ = some_async_work() => {
|
||||
/// println!("operation completed");
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// ```
|
||||
#[macro_export]
|
||||
#[cfg_attr(docsrs, doc(cfg(feature = "macros")))]
|
||||
macro_rules! select {
|
||||
|
@ -125,6 +125,13 @@ impl TcpListener {
|
||||
/// established, the corresponding [`TcpStream`] and the remote peer's
|
||||
/// address will be returned.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If the method is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that no new connections were
|
||||
/// accepted by this method.
|
||||
///
|
||||
/// [`TcpStream`]: struct@crate::net::TcpStream
|
||||
///
|
||||
/// # Examples
|
||||
|
@ -356,6 +356,13 @@ impl TcpStream {
|
||||
/// can be used to concurrently read / write to the same socket on a single
|
||||
/// task without splitting the socket.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read or write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// Concurrently read and write to the stream on the same task without
|
||||
@ -420,6 +427,13 @@ impl TcpStream {
|
||||
/// This function is equivalent to `ready(Interest::READABLE)` and is usually
|
||||
/// paired with `try_read()`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -725,6 +739,13 @@ impl TcpStream {
|
||||
/// This function is equivalent to `ready(Interest::WRITABLE)` and is usually
|
||||
/// paired with `try_write()`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -1152,6 +1173,12 @@ impl TcpStream {
|
||||
split_owned(self)
|
||||
}
|
||||
|
||||
// == Poll IO functions that takes `&self` ==
|
||||
//
|
||||
// To read or write without mutable access to the `UnixStream`, combine the
|
||||
// `poll_read_ready` or `poll_write_ready` methods with the `try_read` or
|
||||
// `try_write` methods.
|
||||
|
||||
pub(crate) fn poll_read_priv(
|
||||
&self,
|
||||
cx: &mut Context<'_>,
|
||||
|
@ -327,6 +327,13 @@ impl UdpSocket {
|
||||
/// false-positive and attempting an operation will return with
|
||||
/// `io::ErrorKind::WouldBlock`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read or write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// Concurrently receive from and send to the socket on the same task
|
||||
@ -390,6 +397,13 @@ impl UdpSocket {
|
||||
/// false-positive and attempting a `try_send()` will return with
|
||||
/// `io::ErrorKind::WouldBlock`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -442,6 +456,12 @@ impl UdpSocket {
|
||||
/// On success, the number of bytes sent is returned, otherwise, the
|
||||
/// encountered error is returned.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `send` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that the message was not sent.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -559,6 +579,13 @@ impl UdpSocket {
|
||||
/// false-positive and attempting a `try_recv()` will return with
|
||||
/// `io::ErrorKind::WouldBlock`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -613,6 +640,13 @@ impl UdpSocket {
|
||||
/// The [`connect`] method will connect this socket to a remote address.
|
||||
/// This method will fail if the socket is not connected.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `recv_from` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, it is guaranteed that no messages were received on this
|
||||
/// socket.
|
||||
///
|
||||
/// [`connect`]: method@Self::connect
|
||||
///
|
||||
/// ```no_run
|
||||
@ -882,6 +916,12 @@ impl UdpSocket {
|
||||
///
|
||||
/// [`ToSocketAddrs`]: crate::net::ToSocketAddrs
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `send_to` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that the message was not sent.
|
||||
///
|
||||
/// # Example
|
||||
///
|
||||
/// ```no_run
|
||||
@ -1005,6 +1045,13 @@ impl UdpSocket {
|
||||
/// size to hold the message bytes. If a message is too long to fit in the
|
||||
/// supplied buffer, excess bytes may be discarded.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `recv_from` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, it is guaranteed that no messages were received on this
|
||||
/// socket.
|
||||
///
|
||||
/// # Example
|
||||
///
|
||||
/// ```no_run
|
||||
|
@ -106,6 +106,13 @@ impl UnixDatagram {
|
||||
/// false-positive and attempting an operation will return with
|
||||
/// `io::ErrorKind::WouldBlock`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read or write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// Concurrently receive from and send to the socket on the same task
|
||||
@ -171,6 +178,13 @@ impl UnixDatagram {
|
||||
/// false-positive and attempting a `try_send()` will return with
|
||||
/// `io::ErrorKind::WouldBlock`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -221,6 +235,13 @@ impl UnixDatagram {
|
||||
/// false-positive and attempting a `try_recv()` will return with
|
||||
/// `io::ErrorKind::WouldBlock`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -490,6 +511,12 @@ impl UnixDatagram {
|
||||
|
||||
/// Sends data on the socket to the socket's peer.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `send` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that the message was not sent.
|
||||
///
|
||||
/// # Examples
|
||||
/// ```
|
||||
/// # use std::error::Error;
|
||||
@ -613,6 +640,13 @@ impl UnixDatagram {
|
||||
|
||||
/// Receives data from the socket.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `recv` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, it is guaranteed that no messages were received on this
|
||||
/// socket.
|
||||
///
|
||||
/// # Examples
|
||||
/// ```
|
||||
/// # use std::error::Error;
|
||||
@ -820,6 +854,12 @@ impl UnixDatagram {
|
||||
|
||||
/// Sends data on the socket to the specified address.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `send_to` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that the message was not sent.
|
||||
///
|
||||
/// # Examples
|
||||
/// ```
|
||||
/// # use std::error::Error;
|
||||
@ -863,6 +903,13 @@ impl UnixDatagram {
|
||||
|
||||
/// Receives data from the socket.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `recv_from` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, it is guaranteed that no messages were received on this
|
||||
/// socket.
|
||||
///
|
||||
/// # Examples
|
||||
/// ```
|
||||
/// # use std::error::Error;
|
||||
|
@ -128,6 +128,13 @@ impl UnixListener {
|
||||
}
|
||||
|
||||
/// Accepts a new incoming connection to this listener.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If the method is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that no new connections were
|
||||
/// accepted by this method.
|
||||
pub async fn accept(&self) -> io::Result<(UnixStream, SocketAddr)> {
|
||||
let (mio, addr) = self
|
||||
.io
|
||||
|
@ -60,6 +60,13 @@ impl UnixStream {
|
||||
/// can be used to concurrently read / write to the same socket on a single
|
||||
/// task without splitting the socket.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read or write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// Concurrently read and write to the stream on the same task without
|
||||
@ -126,6 +133,13 @@ impl UnixStream {
|
||||
/// This function is equivalent to `ready(Interest::READABLE)` and is usually
|
||||
/// paired with `try_read()`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to read that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -435,6 +449,13 @@ impl UnixStream {
|
||||
/// This function is equivalent to `ready(Interest::WRITABLE)` and is usually
|
||||
/// paired with `try_write()`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once a readiness event occurs, the method
|
||||
/// will continue to return immediately until the readiness event is
|
||||
/// consumed by an attempt to write that fails with `WouldBlock` or
|
||||
/// `Poll::Pending`.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```no_run
|
||||
@ -826,14 +847,9 @@ impl AsyncWrite for UnixStream {
|
||||
impl UnixStream {
|
||||
// == Poll IO functions that takes `&self` ==
|
||||
//
|
||||
// They are not public because (taken from the doc of `PollEvented`):
|
||||
//
|
||||
// While `PollEvented` is `Sync` (if the underlying I/O type is `Sync`), the
|
||||
// caller must ensure that there are at most two tasks that use a
|
||||
// `PollEvented` instance concurrently. One for reading and one for writing.
|
||||
// While violating this requirement is "safe" from a Rust memory model point
|
||||
// of view, it will result in unexpected behavior in the form of lost
|
||||
// notifications and tasks hanging.
|
||||
// To read or write without mutable access to the `UnixStream`, combine the
|
||||
// `poll_read_ready` or `poll_write_ready` methods with the `try_read` or
|
||||
// `try_write` methods.
|
||||
|
||||
pub(crate) fn poll_read_priv(
|
||||
&self,
|
||||
|
@ -163,8 +163,16 @@ impl NamedPipeServer {
|
||||
///
|
||||
/// This corresponds to the [`ConnectNamedPipe`] system call.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancellation safe in the sense that if it is used as the
|
||||
/// event in a [`select!`](crate::select) statement and some other branch
|
||||
/// completes first, then no connection events have been lost.
|
||||
///
|
||||
/// [`ConnectNamedPipe`]: https://docs.microsoft.com/en-us/windows/win32/api/namedpipeapi/nf-namedpipeapi-connectnamedpipe
|
||||
///
|
||||
/// # Example
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio::net::windows::named_pipe::ServerOptions;
|
||||
///
|
||||
|
@ -824,6 +824,13 @@ impl<T: Clone> Receiver<T> {
|
||||
/// the channel. A subsequent call to [`recv`] will return this value
|
||||
/// **unless** it has been since overwritten.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `recv` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, it is guaranteed that no messages were received on this
|
||||
/// channel.
|
||||
///
|
||||
/// [`Receiver`]: crate::sync::broadcast::Receiver
|
||||
/// [`recv`]: crate::sync::broadcast::Receiver::recv
|
||||
///
|
||||
|
@ -134,11 +134,16 @@ impl<T> Receiver<T> {
|
||||
///
|
||||
/// If there are no messages in the channel's buffer, but the channel has
|
||||
/// not yet been closed, this method will sleep until a message is sent or
|
||||
/// the channel is closed.
|
||||
/// the channel is closed. Note that if [`close`] is called, but there are
|
||||
/// still outstanding [`Permits`] from before it was closed, the channel is
|
||||
/// not considered closed by `recv` until the permits are released.
|
||||
///
|
||||
/// Note that if [`close`] is called, but there are still outstanding
|
||||
/// [`Permits`] from before it was closed, the channel is not considered
|
||||
/// closed by `recv` until the permits are released.
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `recv` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, it is guaranteed that no messages were received on this
|
||||
/// channel.
|
||||
///
|
||||
/// [`close`]: Self::close
|
||||
/// [`Permits`]: struct@crate::sync::mpsc::Permit
|
||||
@ -335,6 +340,16 @@ impl<T> Sender<T> {
|
||||
/// [`close`]: Receiver::close
|
||||
/// [`Receiver`]: Receiver
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// If `send` is used as the event in a [`tokio::select!`](crate::select)
|
||||
/// statement and some other branch completes first, then it is guaranteed
|
||||
/// that the message was not sent.
|
||||
///
|
||||
/// This channel uses a queue to ensure that calls to `send` and `reserve`
|
||||
/// complete in the order they were requested. Cancelling a call to
|
||||
/// `send` makes you lose your place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// In the following example, each call to `send` will block until the
|
||||
@ -376,6 +391,11 @@ impl<T> Sender<T> {
|
||||
/// This allows the producers to get notified when interest in the produced
|
||||
/// values is canceled and immediately stop doing work.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once the channel is closed, it stays closed
|
||||
/// forever and all future calls to `closed` will return immediately.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -617,6 +637,12 @@ impl<T> Sender<T> {
|
||||
/// [`Permit`]: Permit
|
||||
/// [`send`]: Permit::send
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This channel uses a queue to ensure that calls to `send` and `reserve`
|
||||
/// complete in the order they were requested. Cancelling a call to
|
||||
/// `reserve` makes you lose your place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -666,6 +692,12 @@ impl<T> Sender<T> {
|
||||
/// Dropping the [`OwnedPermit`] without sending a message releases the
|
||||
/// capacity back to the channel.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This channel uses a queue to ensure that calls to `send` and `reserve`
|
||||
/// complete in the order they were requested. Cancelling a call to
|
||||
/// `reserve_owned` makes you lose your place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
/// Sending a message using an [`OwnedPermit`]:
|
||||
/// ```
|
||||
|
@ -82,6 +82,13 @@ impl<T> UnboundedReceiver<T> {
|
||||
/// `None` is returned when all `Sender` halves have dropped, indicating
|
||||
/// that no further values can be sent on the channel.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If `recv` is used as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, it is guaranteed that no messages were received on this
|
||||
/// channel.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -241,6 +248,11 @@ impl<T> UnboundedSender<T> {
|
||||
/// This allows the producers to get notified when interest in the produced
|
||||
/// values is canceled and immediately stop doing work.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once the channel is closed, it stays closed
|
||||
/// forever and all future calls to `closed` will return immediately.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -270,6 +282,7 @@ impl<T> UnboundedSender<T> {
|
||||
pub async fn closed(&self) {
|
||||
self.chan.closed().await
|
||||
}
|
||||
|
||||
/// Checks if the channel has been closed. This happens when the
|
||||
/// [`UnboundedReceiver`] is dropped, or when the
|
||||
/// [`UnboundedReceiver::close`] method is called.
|
||||
|
@ -273,9 +273,15 @@ impl<T: ?Sized> Mutex<T> {
|
||||
}
|
||||
}
|
||||
|
||||
/// Locks this mutex, causing the current task
|
||||
/// to yield until the lock has been acquired.
|
||||
/// When the lock has been acquired, function returns a [`MutexGuard`].
|
||||
/// Locks this mutex, causing the current task to yield until the lock has
|
||||
/// been acquired. When the lock has been acquired, function returns a
|
||||
/// [`MutexGuard`].
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute locks in the order they
|
||||
/// were requested. Cancelling a call to `lock` makes you lose your place in
|
||||
/// the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
@ -305,6 +311,12 @@ impl<T: ?Sized> Mutex<T> {
|
||||
/// method, and the guard will live for the `'static` lifetime, as it keeps
|
||||
/// the `Mutex` alive by holding an `Arc`.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute locks in the order they
|
||||
/// were requested. Cancelling a call to `lock_owned` makes you lose your
|
||||
/// place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
|
@ -246,6 +246,12 @@ impl Notify {
|
||||
///
|
||||
/// [`notify_one()`]: Notify::notify_one
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute notifications in the order
|
||||
/// they were requested. Cancelling a call to `notified` makes you lose your
|
||||
/// place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
|
@ -299,6 +299,12 @@ impl<T: ?Sized> RwLock<T> {
|
||||
/// Returns an RAII guard which will drop this read access of the `RwLock`
|
||||
/// when dropped.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute locks in the order they
|
||||
/// were requested. Cancelling a call to `read` makes you lose your place in
|
||||
/// the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -357,6 +363,12 @@ impl<T: ?Sized> RwLock<T> {
|
||||
/// Returns an RAII guard which will drop this read access of the `RwLock`
|
||||
/// when dropped.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute locks in the order they
|
||||
/// were requested. Cancelling a call to `read_owned` makes you lose your
|
||||
/// place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -501,6 +513,12 @@ impl<T: ?Sized> RwLock<T> {
|
||||
/// Returns an RAII guard which will drop the write access of this `RwLock`
|
||||
/// when dropped.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute locks in the order they
|
||||
/// were requested. Cancelling a call to `write` makes you lose your place
|
||||
/// in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -543,6 +561,12 @@ impl<T: ?Sized> RwLock<T> {
|
||||
/// Returns an RAII guard which will drop the write access of this `RwLock`
|
||||
/// when dropped.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute locks in the order they
|
||||
/// were requested. Cancelling a call to `write_owned` makes you lose your
|
||||
/// place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
|
@ -162,6 +162,12 @@ impl Semaphore {
|
||||
/// Otherwise, this returns a [`SemaphorePermit`] representing the
|
||||
/// acquired permit.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute permits in the order they
|
||||
/// were requested. Cancelling a call to `acquire` makes you lose your place
|
||||
/// in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -198,6 +204,12 @@ impl Semaphore {
|
||||
/// Otherwise, this returns a [`SemaphorePermit`] representing the
|
||||
/// acquired permits.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute permits in the order they
|
||||
/// were requested. Cancelling a call to `acquire_many` makes you lose your
|
||||
/// place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -302,6 +314,12 @@ impl Semaphore {
|
||||
/// Otherwise, this returns a [`OwnedSemaphorePermit`] representing the
|
||||
/// acquired permit.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute permits in the order they
|
||||
/// were requested. Cancelling a call to `acquire_owned` makes you lose your
|
||||
/// place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
@ -346,6 +364,12 @@ impl Semaphore {
|
||||
/// Otherwise, this returns a [`OwnedSemaphorePermit`] representing the
|
||||
/// acquired permit.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method uses a queue to fairly distribute permits in the order they
|
||||
/// were requested. Cancelling a call to `acquire_many_owned` makes you lose
|
||||
/// your place in the queue.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
|
@ -261,6 +261,13 @@ impl<T> Receiver<T> {
|
||||
///
|
||||
/// This method returns an error if and only if the [`Sender`] is dropped.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. If you use it as the event in a
|
||||
/// [`tokio::select!`](crate::select) statement and some other branch
|
||||
/// completes first, then it is guaranteed that no values have been marked
|
||||
/// seen by this call to `changed`.
|
||||
///
|
||||
/// [`Sender`]: struct@Sender
|
||||
///
|
||||
/// # Examples
|
||||
@ -418,6 +425,11 @@ impl<T> Sender<T> {
|
||||
/// This allows the producer to get notified when interest in the produced
|
||||
/// values is canceled and immediately stop doing work.
|
||||
///
|
||||
/// # Cancel safety
|
||||
///
|
||||
/// This method is cancel safe. Once the channel is closed, it stays closed
|
||||
/// forever and all future calls to `closed` will return immediately.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
|
Loading…
x
Reference in New Issue
Block a user