aboutsummaryrefslogtreecommitdiff
path: root/src/process/mod.rs
diff options
context:
space:
mode:
Diffstat (limited to 'src/process/mod.rs')
-rw-r--r--src/process/mod.rs318
1 files changed, 240 insertions, 78 deletions
diff --git a/src/process/mod.rs b/src/process/mod.rs
index e04a435..ad64371 100644
--- a/src/process/mod.rs
+++ b/src/process/mod.rs
@@ -18,16 +18,15 @@
//!
//! #[tokio::main]
//! async fn main() -> Result<(), Box<dyn std::error::Error>> {
-//! // The usage is the same as with the standard library's `Command` type, however the value
-//! // returned from `spawn` is a `Result` containing a `Future`.
-//! let child = Command::new("echo").arg("hello").arg("world")
-//! .spawn();
+//! // The usage is similar as with the standard library's `Command` type
+//! let mut child = Command::new("echo")
+//! .arg("hello")
+//! .arg("world")
+//! .spawn()
+//! .expect("failed to spawn");
//!
-//! // Make sure our child succeeded in spawning and process the result
-//! let future = child.expect("failed to spawn");
-//!
-//! // Await until the future (and the command) completes
-//! let status = future.await?;
+//! // Await until the command completes
+//! let status = child.wait().await?;
//! println!("the command exited with: {}", status);
//! Ok(())
//! }
@@ -83,8 +82,8 @@
//!
//! // Ensure the child process is spawned in the runtime so it can
//! // make progress on its own while we await for any output.
-//! tokio::spawn(async {
-//! let status = child.await
+//! tokio::spawn(async move {
+//! let status = child.wait().await
//! .expect("child process encountered an error");
//!
//! println!("child status was: {}", status);
@@ -100,27 +99,52 @@
//!
//! # Caveats
//!
+//! ## Dropping/Cancellation
+//!
//! Similar to the behavior to the standard library, and unlike the futures
//! paradigm of dropping-implies-cancellation, a spawned process will, by
//! default, continue to execute even after the `Child` handle has been dropped.
//!
-//! The `Command::kill_on_drop` method can be used to modify this behavior
+//! The [`Command::kill_on_drop`] method can be used to modify this behavior
//! and kill the child process if the `Child` wrapper is dropped before it
//! has exited.
//!
+//! ## Unix Processes
+//!
+//! On Unix platforms processes must be "reaped" by their parent process after
+//! they have exited in order to release all OS resources. A child process which
+//! has exited, but has not yet been reaped by its parent is considered a "zombie"
+//! process. Such processes continue to count against limits imposed by the system,
+//! and having too many zombie processes present can prevent additional processes
+//! from being spawned.
+//!
+//! The tokio runtime will, on a best-effort basis, attempt to reap and clean up
+//! any process which it has spawned. No additional guarantees are made with regards
+//! how quickly or how often this procedure will take place.
+//!
+//! It is recommended to avoid dropping a [`Child`] process handle before it has been
+//! fully `await`ed if stricter cleanup guarantees are required.
+//!
//! [`Command`]: crate::process::Command
+//! [`Command::kill_on_drop`]: crate::process::Command::kill_on_drop
+//! [`Child`]: crate::process::Child
#[path = "unix/mod.rs"]
#[cfg(unix)]
mod imp;
+#[cfg(unix)]
+pub(crate) mod unix {
+ pub(crate) use super::imp::*;
+}
+
#[path = "windows.rs"]
#[cfg(windows)]
mod imp;
mod kill;
-use crate::io::{AsyncRead, AsyncWrite};
+use crate::io::{AsyncRead, AsyncWrite, ReadBuf};
use crate::process::kill::Kill;
use std::ffi::OsStr;
@@ -450,6 +474,26 @@ impl Command {
/// By default, this value is assumed to be `false`, meaning the next spawned
/// process will not be killed on drop, similar to the behavior of the standard
/// library.
+ ///
+ /// # Caveats
+ ///
+ /// On Unix platforms processes must be "reaped" by their parent process after
+ /// they have exited in order to release all OS resources. A child process which
+ /// has exited, but has not yet been reaped by its parent is considered a "zombie"
+ /// process. Such processes continue to count against limits imposed by the system,
+ /// and having too many zombie processes present can prevent additional processes
+ /// from being spawned.
+ ///
+ /// Although issuing a `kill` signal to the child process is a synchronous
+ /// operation, the resulting zombie process cannot be `.await`ed inside of the
+ /// destructor to avoid blocking other tasks. The tokio runtime will, on a
+ /// best-effort basis, attempt to reap and clean up such processes in the
+ /// background, but makes no additional guarantees are made with regards
+ /// how quickly or how often this procedure will take place.
+ ///
+ /// If stronger guarantees are required, it is recommended to avoid dropping
+ /// a [`Child`] handle where possible, and instead utilize `child.wait().await`
+ /// or `child.kill().await` where possible.
pub fn kill_on_drop(&mut self, kill_on_drop: bool) -> &mut Command {
self.kill_on_drop = kill_on_drop;
self
@@ -534,16 +578,6 @@ impl Command {
/// All I/O this child does will be associated with the current default
/// event loop.
///
- /// # Caveats
- ///
- /// Similar to the behavior to the standard library, and unlike the futures
- /// paradigm of dropping-implies-cancellation, the spawned process will, by
- /// default, continue to execute even after the `Child` handle has been dropped.
- ///
- /// The `Command::kill_on_drop` method can be used to modify this behavior
- /// and kill the child process if the `Child` wrapper is dropped before it
- /// has exited.
- ///
/// # Examples
///
/// Basic usage:
@@ -555,16 +589,55 @@ impl Command {
/// Command::new("ls")
/// .spawn()
/// .expect("ls command failed to start")
+ /// .wait()
/// .await
/// .expect("ls command failed to run")
/// }
/// ```
+ ///
+ /// # Caveats
+ ///
+ /// ## Dropping/Cancellation
+ ///
+ /// Similar to the behavior to the standard library, and unlike the futures
+ /// paradigm of dropping-implies-cancellation, a spawned process will, by
+ /// default, continue to execute even after the `Child` handle has been dropped.
+ ///
+ /// The [`Command::kill_on_drop`] method can be used to modify this behavior
+ /// and kill the child process if the `Child` wrapper is dropped before it
+ /// has exited.
+ ///
+ /// ## Unix Processes
+ ///
+ /// On Unix platforms processes must be "reaped" by their parent process after
+ /// they have exited in order to release all OS resources. A child process which
+ /// has exited, but has not yet been reaped by its parent is considered a "zombie"
+ /// process. Such processes continue to count against limits imposed by the system,
+ /// and having too many zombie processes present can prevent additional processes
+ /// from being spawned.
+ ///
+ /// The tokio runtime will, on a best-effort basis, attempt to reap and clean up
+ /// any process which it has spawned. No additional guarantees are made with regards
+ /// how quickly or how often this procedure will take place.
+ ///
+ /// It is recommended to avoid dropping a [`Child`] process handle before it has been
+ /// fully `await`ed if stricter cleanup guarantees are required.
+ ///
+ /// [`Command`]: crate::process::Command
+ /// [`Command::kill_on_drop`]: crate::process::Command::kill_on_drop
+ /// [`Child`]: crate::process::Child
+ ///
+ /// # Errors
+ ///
+ /// On Unix platforms this method will fail with `std::io::ErrorKind::WouldBlock`
+ /// if the system process limit is reached (which includes other applications
+ /// running on the system).
pub fn spawn(&mut self) -> io::Result<Child> {
imp::spawn_child(&mut self.std).map(|spawned_child| Child {
- child: ChildDropGuard {
+ child: FusedChild::Child(ChildDropGuard {
inner: spawned_child.child,
kill_on_drop: self.kill_on_drop,
- },
+ }),
stdin: spawned_child.stdin.map(|inner| ChildStdin { inner }),
stdout: spawned_child.stdout.map(|inner| ChildStdout { inner }),
stderr: spawned_child.stderr.map(|inner| ChildStderr { inner }),
@@ -581,14 +654,20 @@ impl Command {
/// All I/O this child does will be associated with the current default
/// event loop.
///
- /// If this future is dropped before the future resolves, then
- /// the child will be killed, if it was spawned.
+ /// The destructor of the future returned by this function will kill
+ /// the child if [`kill_on_drop`] is set to true.
+ ///
+ /// [`kill_on_drop`]: fn@Self::kill_on_drop
///
/// # Errors
///
/// This future will return an error if the child process cannot be spawned
/// or if there is an error while awaiting its status.
///
+ /// On Unix platforms this method will fail with `std::io::ErrorKind::WouldBlock`
+ /// if the system process limit is reached (which includes other applications
+ /// running on the system).
+ ///
/// # Examples
///
/// Basic usage:
@@ -602,6 +681,7 @@ impl Command {
/// .await
/// .expect("ls command failed to run")
/// }
+ /// ```
pub fn status(&mut self) -> impl Future<Output = io::Result<ExitStatus>> {
let child = self.spawn();
@@ -615,7 +695,7 @@ impl Command {
child.stdout.take();
child.stderr.take();
- child.await
+ child.wait().await
}
}
@@ -637,9 +717,19 @@ impl Command {
/// All I/O this child does will be associated with the current default
/// event loop.
///
- /// If this future is dropped before the future resolves, then
- /// the child will be killed, if it was spawned.
+ /// The destructor of the future returned by this function will kill
+ /// the child if [`kill_on_drop`] is set to true.
+ ///
+ /// [`kill_on_drop`]: fn@Self::kill_on_drop
+ ///
+ /// # Errors
+ ///
+ /// This future will return an error if the child process cannot be spawned
+ /// or if there is an error while awaiting its status.
///
+ /// On Unix platforms this method will fail with `std::io::ErrorKind::WouldBlock`
+ /// if the system process limit is reached (which includes other applications
+ /// running on the system).
/// # Examples
///
/// Basic usage:
@@ -654,6 +744,7 @@ impl Command {
/// .expect("ls command failed to run");
/// println!("stderr of ls: {:?}", output.stderr);
/// }
+ /// ```
pub fn output(&mut self) -> impl Future<Output = io::Result<Output>> {
self.std.stdout(Stdio::piped());
self.std.stderr(Stdio::piped());
@@ -725,12 +816,16 @@ where
}
}
+/// Keeps track of the exit status of a child process without worrying about
+/// polling the underlying futures even after they have completed.
+#[derive(Debug)]
+enum FusedChild {
+ Child(ChildDropGuard<imp::Child>),
+ Done(ExitStatus),
+}
+
/// Representation of a child process spawned onto an event loop.
///
-/// This type is also a future which will yield the `ExitStatus` of the
-/// underlying child process. A `Child` here also provides access to information
-/// like the OS-assigned identifier and the stdio streams.
-///
/// # Caveats
/// Similar to the behavior to the standard library, and unlike the futures
/// paradigm of dropping-implies-cancellation, a spawned process will, by
@@ -739,10 +834,9 @@ where
/// The `Command::kill_on_drop` method can be used to modify this behavior
/// and kill the child process if the `Child` wrapper is dropped before it
/// has exited.
-#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct Child {
- child: ChildDropGuard<imp::Child>,
+ child: FusedChild,
/// The handle for writing to the child's standard input (stdin), if it has
/// been captured.
@@ -758,34 +852,120 @@ pub struct Child {
}
impl Child {
- /// Returns the OS-assigned process identifier associated with this child.
- pub fn id(&self) -> u32 {
- self.child.inner.id()
+ /// Returns the OS-assigned process identifier associated with this child
+ /// while it is still running.
+ ///
+ /// Once the child has been polled to completion this will return `None`.
+ /// This is done to avoid confusion on platforms like Unix where the OS
+ /// identifier could be reused once the process has completed.
+ pub fn id(&self) -> Option<u32> {
+ match &self.child {
+ FusedChild::Child(child) => Some(child.inner.id()),
+ FusedChild::Done(_) => None,
+ }
+ }
+
+ /// Attempts to force the child to exit, but does not wait for the request
+ /// to take effect.
+ ///
+ /// On Unix platforms, this is the equivalent to sending a SIGKILL. Note
+ /// that on Unix platforms it is possible for a zombie process to remain
+ /// after a kill is sent; to avoid this, the caller should ensure that either
+ /// `child.wait().await` or `child.try_wait()` is invoked successfully.
+ pub fn start_kill(&mut self) -> io::Result<()> {
+ match &mut self.child {
+ FusedChild::Child(child) => child.kill(),
+ FusedChild::Done(_) => Err(io::Error::new(
+ io::ErrorKind::InvalidInput,
+ "invalid argument: can't kill an exited process",
+ )),
+ }
}
/// Forces the child to exit.
///
/// This is equivalent to sending a SIGKILL on unix platforms.
- pub fn kill(&mut self) -> io::Result<()> {
- self.child.kill()
- }
-
- #[doc(hidden)]
- #[deprecated(note = "please use `child.stdin` instead")]
- pub fn stdin(&mut self) -> &mut Option<ChildStdin> {
- &mut self.stdin
+ ///
+ /// If the child has to be killed remotely, it is possible to do it using
+ /// a combination of the select! macro and a oneshot channel. In the following
+ /// example, the child will run until completion unless a message is sent on
+ /// the oneshot channel. If that happens, the child is killed immediately
+ /// using the `.kill()` method.
+ ///
+ /// ```no_run
+ /// use tokio::process::Command;
+ /// use tokio::sync::oneshot::channel;
+ ///
+ /// #[tokio::main]
+ /// async fn main() {
+ /// let (send, recv) = channel::<()>();
+ /// let mut child = Command::new("sleep").arg("1").spawn().unwrap();
+ /// tokio::spawn(async move { send.send(()) });
+ /// tokio::select! {
+ /// _ = child.wait() => {}
+ /// _ = recv => child.kill().await.expect("kill failed"),
+ /// }
+ /// }
+ /// ```
+ pub async fn kill(&mut self) -> io::Result<()> {
+ self.start_kill()?;
+ self.wait().await?;
+ Ok(())
}
- #[doc(hidden)]
- #[deprecated(note = "please use `child.stdout` instead")]
- pub fn stdout(&mut self) -> &mut Option<ChildStdout> {
- &mut self.stdout
+ /// Waits for the child to exit completely, returning the status that it
+ /// exited with. This function will continue to have the same return value
+ /// after it has been called at least once.
+ ///
+ /// The stdin handle to the child process, if any, will be closed
+ /// before waiting. This helps avoid deadlock: it ensures that the
+ /// child does not block waiting for input from the parent, while
+ /// the parent waits for the child to exit.
+ pub async fn wait(&mut self) -> io::Result<ExitStatus> {
+ match &mut self.child {
+ FusedChild::Done(exit) => Ok(*exit),
+ FusedChild::Child(child) => {
+ let ret = child.await;
+
+ if let Ok(exit) = ret {
+ self.child = FusedChild::Done(exit);
+ }
+
+ ret
+ }
+ }
}
- #[doc(hidden)]
- #[deprecated(note = "please use `child.stderr` instead")]
- pub fn stderr(&mut self) -> &mut Option<ChildStderr> {
- &mut self.stderr
+ /// Attempts to collect the exit status of the child if it has already
+ /// exited.
+ ///
+ /// This function will not block the calling thread and will only
+ /// check to see if the child process has exited or not. If the child has
+ /// exited then on Unix the process ID is reaped. This function is
+ /// guaranteed to repeatedly return a successful exit status so long as the
+ /// child has already exited.
+ ///
+ /// If the child has exited, then `Ok(Some(status))` is returned. If the
+ /// exit status is not available at this time then `Ok(None)` is returned.
+ /// If an error occurs, then that error is returned.
+ ///
+ /// Note that unlike `wait`, this function will not attempt to drop stdin,
+ /// nor will it wake the current task if the child exits.
+ pub fn try_wait(&mut self) -> io::Result<Option<ExitStatus>> {
+ match &mut self.child {
+ FusedChild::Done(exit) => Ok(Some(*exit)),
+ FusedChild::Child(guard) => {
+ let ret = guard.inner.try_wait();
+
+ if let Ok(Some(exit)) = ret {
+ // Avoid the overhead of trying to kill a reaped process
+ guard.kill_on_drop = false;
+ self.child = FusedChild::Done(exit);
+ }
+
+ ret
+ }
+ }
}
/// Returns a future that will resolve to an `Output`, containing the exit
@@ -819,7 +999,7 @@ impl Child {
let stdout_fut = read_to_end(self.stdout.take());
let stderr_fut = read_to_end(self.stderr.take());
- let (status, stdout, stderr) = try_join3(self, stdout_fut, stderr_fut).await?;
+ let (status, stdout, stderr) = try_join3(self.wait(), stdout_fut, stderr_fut).await?;
Ok(Output {
status,
@@ -829,14 +1009,6 @@ impl Child {
}
}
-impl Future for Child {
- type Output = io::Result<ExitStatus>;
-
- fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
- Pin::new(&mut self.child).poll(cx)
- }
-}
-
/// The standard input stream for spawned children.
///
/// This type implements the `AsyncWrite` trait to pass data to the stdin handle of
@@ -883,31 +1055,21 @@ impl AsyncWrite for ChildStdin {
}
impl AsyncRead for ChildStdout {
- unsafe fn prepare_uninitialized_buffer(&self, _buf: &mut [std::mem::MaybeUninit<u8>]) -> bool {
- // https://github.com/rust-lang/rust/blob/09c817eeb29e764cfc12d0a8d94841e3ffe34023/src/libstd/process.rs#L314
- false
- }
-
fn poll_read(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
- buf: &mut [u8],
- ) -> Poll<io::Result<usize>> {
+ buf: &mut ReadBuf<'_>,
+ ) -> Poll<io::Result<()>> {
Pin::new(&mut self.inner).poll_read(cx, buf)
}
}
impl AsyncRead for ChildStderr {
- unsafe fn prepare_uninitialized_buffer(&self, _buf: &mut [std::mem::MaybeUninit<u8>]) -> bool {
- // https://github.com/rust-lang/rust/blob/09c817eeb29e764cfc12d0a8d94841e3ffe34023/src/libstd/process.rs#L375
- false
- }
-
fn poll_read(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
- buf: &mut [u8],
- ) -> Poll<io::Result<usize>> {
+ buf: &mut ReadBuf<'_>,
+ ) -> Poll<io::Result<()>> {
Pin::new(&mut self.inner).poll_read(cx, buf)
}
}