mirror of
https://github.com/tokio-rs/tokio.git
synced 2025-09-25 12:00:35 +00:00
process: add Child::{wait,try_wait} (#2796)
* add Child::try_wait to mirror the std API * replace Future impl on Child with `.wait()` method to bring our APIs closer to those in std and it allow us to internally fuse the future so that repeated calls to `wait` result in the same value (similar to std) without forcing the caller to fuse the outer future * Also change `Child::id` to return an Option result to avoid allowing the caller to accidentally use the pid on Unix systems after the child has been reaped * Also remove deprecated Child methods
This commit is contained in:
parent
d74eabc7d7
commit
842d5565bd
@ -72,7 +72,7 @@ async fn feed_cat(mut cat: Child, n: usize) -> io::Result<ExitStatus> {
|
|||||||
};
|
};
|
||||||
|
|
||||||
// Compose reading and writing concurrently.
|
// Compose reading and writing concurrently.
|
||||||
future::join3(write, read, cat)
|
future::join3(write, read, cat.wait())
|
||||||
.map(|(_, _, status)| status)
|
.map(|(_, _, status)| status)
|
||||||
.await
|
.await
|
||||||
}
|
}
|
||||||
@ -125,3 +125,26 @@ async fn status_closes_any_pipes() {
|
|||||||
|
|
||||||
assert_ok!(child.await);
|
assert_ok!(child.await);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
async fn try_wait() {
|
||||||
|
let mut child = cat().spawn().unwrap();
|
||||||
|
|
||||||
|
let id = child.id().expect("missing id");
|
||||||
|
assert!(id > 0);
|
||||||
|
|
||||||
|
assert_eq!(None, assert_ok!(child.try_wait()));
|
||||||
|
|
||||||
|
// Drop the child's stdio handles so it can terminate
|
||||||
|
drop(child.stdin.take());
|
||||||
|
drop(child.stderr.take());
|
||||||
|
drop(child.stdout.take());
|
||||||
|
|
||||||
|
assert_ok!(child.wait().await);
|
||||||
|
|
||||||
|
// test that the `.try_wait()` method is fused just like the stdlib
|
||||||
|
assert!(assert_ok!(child.try_wait()).unwrap().success());
|
||||||
|
|
||||||
|
// Can't get id after process has exited
|
||||||
|
assert_eq!(child.id(), None);
|
||||||
|
}
|
||||||
|
@ -63,9 +63,7 @@ process = [
|
|||||||
"mio-named-pipes",
|
"mio-named-pipes",
|
||||||
"signal",
|
"signal",
|
||||||
"winapi/consoleapi",
|
"winapi/consoleapi",
|
||||||
"winapi/minwindef",
|
|
||||||
"winapi/threadpoollegacyapiset",
|
"winapi/threadpoollegacyapiset",
|
||||||
"winapi/winerror",
|
|
||||||
]
|
]
|
||||||
# Includes basic task execution capabilities
|
# Includes basic task execution capabilities
|
||||||
rt-core = ["slab"]
|
rt-core = ["slab"]
|
||||||
@ -81,7 +79,6 @@ signal = [
|
|||||||
"mio-uds",
|
"mio-uds",
|
||||||
"signal-hook-registry",
|
"signal-hook-registry",
|
||||||
"winapi/consoleapi",
|
"winapi/consoleapi",
|
||||||
"winapi/minwindef",
|
|
||||||
]
|
]
|
||||||
stream = ["futures-core"]
|
stream = ["futures-core"]
|
||||||
sync = ["fnv"]
|
sync = ["fnv"]
|
||||||
|
@ -18,16 +18,15 @@
|
|||||||
//!
|
//!
|
||||||
//! #[tokio::main]
|
//! #[tokio::main]
|
||||||
//! async fn main() -> Result<(), Box<dyn std::error::Error>> {
|
//! 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
|
//! // The usage is similar as with the standard library's `Command` type
|
||||||
//! // returned from `spawn` is a `Result` containing a `Future`.
|
//! let mut child = Command::new("echo")
|
||||||
//! let child = Command::new("echo").arg("hello").arg("world")
|
//! .arg("hello")
|
||||||
//! .spawn();
|
//! .arg("world")
|
||||||
|
//! .spawn()
|
||||||
|
//! .expect("failed to spawn");
|
||||||
//!
|
//!
|
||||||
//! // Make sure our child succeeded in spawning and process the result
|
//! // Await until the command completes
|
||||||
//! let future = child.expect("failed to spawn");
|
//! let status = child.wait().await?;
|
||||||
//!
|
|
||||||
//! // Await until the future (and the command) completes
|
|
||||||
//! let status = future.await?;
|
|
||||||
//! println!("the command exited with: {}", status);
|
//! println!("the command exited with: {}", status);
|
||||||
//! Ok(())
|
//! Ok(())
|
||||||
//! }
|
//! }
|
||||||
@ -83,8 +82,8 @@
|
|||||||
//!
|
//!
|
||||||
//! // Ensure the child process is spawned in the runtime so it can
|
//! // Ensure the child process is spawned in the runtime so it can
|
||||||
//! // make progress on its own while we await for any output.
|
//! // make progress on its own while we await for any output.
|
||||||
//! tokio::spawn(async {
|
//! tokio::spawn(async move {
|
||||||
//! let status = child.await
|
//! let status = child.wait().await
|
||||||
//! .expect("child process encountered an error");
|
//! .expect("child process encountered an error");
|
||||||
//!
|
//!
|
||||||
//! println!("child status was: {}", status);
|
//! println!("child status was: {}", status);
|
||||||
@ -555,16 +554,17 @@ impl Command {
|
|||||||
/// Command::new("ls")
|
/// Command::new("ls")
|
||||||
/// .spawn()
|
/// .spawn()
|
||||||
/// .expect("ls command failed to start")
|
/// .expect("ls command failed to start")
|
||||||
|
/// .wait()
|
||||||
/// .await
|
/// .await
|
||||||
/// .expect("ls command failed to run")
|
/// .expect("ls command failed to run")
|
||||||
/// }
|
/// }
|
||||||
/// ```
|
/// ```
|
||||||
pub fn spawn(&mut self) -> io::Result<Child> {
|
pub fn spawn(&mut self) -> io::Result<Child> {
|
||||||
imp::spawn_child(&mut self.std).map(|spawned_child| Child {
|
imp::spawn_child(&mut self.std).map(|spawned_child| Child {
|
||||||
child: ChildDropGuard {
|
child: FusedChild::Child(ChildDropGuard {
|
||||||
inner: spawned_child.child,
|
inner: spawned_child.child,
|
||||||
kill_on_drop: self.kill_on_drop,
|
kill_on_drop: self.kill_on_drop,
|
||||||
},
|
}),
|
||||||
stdin: spawned_child.stdin.map(|inner| ChildStdin { inner }),
|
stdin: spawned_child.stdin.map(|inner| ChildStdin { inner }),
|
||||||
stdout: spawned_child.stdout.map(|inner| ChildStdout { inner }),
|
stdout: spawned_child.stdout.map(|inner| ChildStdout { inner }),
|
||||||
stderr: spawned_child.stderr.map(|inner| ChildStderr { inner }),
|
stderr: spawned_child.stderr.map(|inner| ChildStderr { inner }),
|
||||||
@ -615,7 +615,7 @@ impl Command {
|
|||||||
child.stdout.take();
|
child.stdout.take();
|
||||||
child.stderr.take();
|
child.stderr.take();
|
||||||
|
|
||||||
child.await
|
child.wait().await
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -725,12 +725,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.
|
/// 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
|
/// # Caveats
|
||||||
/// Similar to the behavior to the standard library, and unlike the futures
|
/// Similar to the behavior to the standard library, and unlike the futures
|
||||||
/// paradigm of dropping-implies-cancellation, a spawned process will, by
|
/// paradigm of dropping-implies-cancellation, a spawned process will, by
|
||||||
@ -739,10 +743,9 @@ where
|
|||||||
/// 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
|
/// and kill the child process if the `Child` wrapper is dropped before it
|
||||||
/// has exited.
|
/// has exited.
|
||||||
#[must_use = "futures do nothing unless polled"]
|
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
pub struct Child {
|
pub struct Child {
|
||||||
child: ChildDropGuard<imp::Child>,
|
child: FusedChild,
|
||||||
|
|
||||||
/// The handle for writing to the child's standard input (stdin), if it has
|
/// The handle for writing to the child's standard input (stdin), if it has
|
||||||
/// been captured.
|
/// been captured.
|
||||||
@ -758,9 +761,17 @@ pub struct Child {
|
|||||||
}
|
}
|
||||||
|
|
||||||
impl Child {
|
impl Child {
|
||||||
/// Returns the OS-assigned process identifier associated with this child.
|
/// Returns the OS-assigned process identifier associated with this child
|
||||||
pub fn id(&self) -> u32 {
|
/// while it is still running.
|
||||||
self.child.inner.id()
|
///
|
||||||
|
/// 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,
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Forces the child to exit.
|
/// Forces the child to exit.
|
||||||
@ -783,35 +794,77 @@ impl Child {
|
|||||||
/// let mut child = Command::new("sleep").arg("1").spawn().unwrap();
|
/// let mut child = Command::new("sleep").arg("1").spawn().unwrap();
|
||||||
/// tokio::spawn(async move { send.send(()) });
|
/// tokio::spawn(async move { send.send(()) });
|
||||||
/// tokio::select! {
|
/// tokio::select! {
|
||||||
/// _ = &mut child => {}
|
/// _ = child.wait() => {}
|
||||||
/// _ = recv => {
|
/// _ = recv => {
|
||||||
/// &mut child.kill();
|
/// child.kill().expect("kill failed");
|
||||||
/// // NB: await the child here to avoid a zombie process on Unix platforms
|
/// // NB: await the child here to avoid a zombie process on Unix platforms
|
||||||
/// child.await.unwrap();
|
/// child.wait().await.unwrap();
|
||||||
/// }
|
/// }
|
||||||
/// }
|
/// }
|
||||||
/// }
|
/// }
|
||||||
|
|
||||||
pub fn kill(&mut self) -> io::Result<()> {
|
pub fn kill(&mut self) -> io::Result<()> {
|
||||||
self.child.kill()
|
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",
|
||||||
|
)),
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[doc(hidden)]
|
/// Waits for the child to exit completely, returning the status that it
|
||||||
#[deprecated(note = "please use `child.stdin` instead")]
|
/// exited with. This function will continue to have the same return value
|
||||||
pub fn stdin(&mut self) -> &mut Option<ChildStdin> {
|
/// after it has been called at least once.
|
||||||
&mut self.stdin
|
///
|
||||||
|
/// 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)]
|
/// Attempts to collect the exit status of the child if it has already
|
||||||
#[deprecated(note = "please use `child.stdout` instead")]
|
/// exited.
|
||||||
pub fn stdout(&mut self) -> &mut Option<ChildStdout> {
|
///
|
||||||
&mut self.stdout
|
/// 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();
|
||||||
|
|
||||||
#[doc(hidden)]
|
if let Ok(Some(exit)) = ret {
|
||||||
#[deprecated(note = "please use `child.stderr` instead")]
|
// Avoid the overhead of trying to kill a reaped process
|
||||||
pub fn stderr(&mut self) -> &mut Option<ChildStderr> {
|
guard.kill_on_drop = false;
|
||||||
&mut self.stderr
|
self.child = FusedChild::Done(exit);
|
||||||
|
}
|
||||||
|
|
||||||
|
ret
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns a future that will resolve to an `Output`, containing the exit
|
/// Returns a future that will resolve to an `Output`, containing the exit
|
||||||
@ -845,7 +898,7 @@ impl Child {
|
|||||||
let stdout_fut = read_to_end(self.stdout.take());
|
let stdout_fut = read_to_end(self.stdout.take());
|
||||||
let stderr_fut = read_to_end(self.stderr.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 {
|
Ok(Output {
|
||||||
status,
|
status,
|
||||||
@ -855,14 +908,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.
|
/// The standard input stream for spawned children.
|
||||||
///
|
///
|
||||||
/// This type implements the `AsyncWrite` trait to pass data to the stdin handle of
|
/// This type implements the `AsyncWrite` trait to pass data to the stdin handle of
|
||||||
|
@ -117,6 +117,10 @@ impl Child {
|
|||||||
pub(crate) fn id(&self) -> u32 {
|
pub(crate) fn id(&self) -> u32 {
|
||||||
self.inner.id()
|
self.inner.id()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub(crate) fn try_wait(&mut self) -> io::Result<Option<ExitStatus>> {
|
||||||
|
self.inner.inner_mut().try_wait()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Kill for Child {
|
impl Kill for Child {
|
||||||
|
@ -63,7 +63,7 @@ where
|
|||||||
self.inner.as_ref().expect("inner has gone away")
|
self.inner.as_ref().expect("inner has gone away")
|
||||||
}
|
}
|
||||||
|
|
||||||
fn inner_mut(&mut self) -> &mut W {
|
pub(crate) fn inner_mut(&mut self) -> &mut W {
|
||||||
self.inner.as_mut().expect("inner has gone away")
|
self.inner.as_mut().expect("inner has gone away")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -24,20 +24,15 @@ use mio_named_pipes::NamedPipe;
|
|||||||
use std::fmt;
|
use std::fmt;
|
||||||
use std::future::Future;
|
use std::future::Future;
|
||||||
use std::io;
|
use std::io;
|
||||||
use std::os::windows::prelude::*;
|
use std::os::windows::prelude::{AsRawHandle, FromRawHandle, IntoRawHandle};
|
||||||
use std::os::windows::process::ExitStatusExt;
|
|
||||||
use std::pin::Pin;
|
use std::pin::Pin;
|
||||||
use std::process::{Child as StdChild, Command as StdCommand, ExitStatus};
|
use std::process::{Child as StdChild, Command as StdCommand, ExitStatus};
|
||||||
use std::ptr;
|
use std::ptr;
|
||||||
use std::task::Context;
|
use std::task::Context;
|
||||||
use std::task::Poll;
|
use std::task::Poll;
|
||||||
use winapi::shared::minwindef::FALSE;
|
|
||||||
use winapi::shared::winerror::WAIT_TIMEOUT;
|
|
||||||
use winapi::um::handleapi::INVALID_HANDLE_VALUE;
|
use winapi::um::handleapi::INVALID_HANDLE_VALUE;
|
||||||
use winapi::um::processthreadsapi::GetExitCodeProcess;
|
|
||||||
use winapi::um::synchapi::WaitForSingleObject;
|
|
||||||
use winapi::um::threadpoollegacyapiset::UnregisterWaitEx;
|
use winapi::um::threadpoollegacyapiset::UnregisterWaitEx;
|
||||||
use winapi::um::winbase::{RegisterWaitForSingleObject, INFINITE, WAIT_OBJECT_0};
|
use winapi::um::winbase::{RegisterWaitForSingleObject, INFINITE};
|
||||||
use winapi::um::winnt::{BOOLEAN, HANDLE, PVOID, WT_EXECUTEINWAITTHREAD, WT_EXECUTEONLYONCE};
|
use winapi::um::winnt::{BOOLEAN, HANDLE, PVOID, WT_EXECUTEINWAITTHREAD, WT_EXECUTEONLYONCE};
|
||||||
|
|
||||||
#[must_use = "futures do nothing unless polled"]
|
#[must_use = "futures do nothing unless polled"]
|
||||||
@ -86,6 +81,10 @@ impl Child {
|
|||||||
pub(crate) fn id(&self) -> u32 {
|
pub(crate) fn id(&self) -> u32 {
|
||||||
self.child.id()
|
self.child.id()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub(crate) fn try_wait(&mut self) -> io::Result<Option<ExitStatus>> {
|
||||||
|
self.child.try_wait()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Kill for Child {
|
impl Kill for Child {
|
||||||
@ -106,11 +105,11 @@ impl Future for Child {
|
|||||||
Poll::Ready(Err(_)) => panic!("should not be canceled"),
|
Poll::Ready(Err(_)) => panic!("should not be canceled"),
|
||||||
Poll::Pending => return Poll::Pending,
|
Poll::Pending => return Poll::Pending,
|
||||||
}
|
}
|
||||||
let status = try_wait(&inner.child)?.expect("not ready yet");
|
let status = inner.try_wait()?.expect("not ready yet");
|
||||||
return Poll::Ready(Ok(status));
|
return Poll::Ready(Ok(status));
|
||||||
}
|
}
|
||||||
|
|
||||||
if let Some(e) = try_wait(&inner.child)? {
|
if let Some(e) = inner.try_wait()? {
|
||||||
return Poll::Ready(Ok(e));
|
return Poll::Ready(Ok(e));
|
||||||
}
|
}
|
||||||
let (tx, rx) = oneshot::channel();
|
let (tx, rx) = oneshot::channel();
|
||||||
@ -157,23 +156,6 @@ unsafe extern "system" fn callback(ptr: PVOID, _timer_fired: BOOLEAN) {
|
|||||||
let _ = complete.take().unwrap().send(());
|
let _ = complete.take().unwrap().send(());
|
||||||
}
|
}
|
||||||
|
|
||||||
pub(crate) fn try_wait(child: &StdChild) -> io::Result<Option<ExitStatus>> {
|
|
||||||
unsafe {
|
|
||||||
match WaitForSingleObject(child.as_raw_handle(), 0) {
|
|
||||||
WAIT_OBJECT_0 => {}
|
|
||||||
WAIT_TIMEOUT => return Ok(None),
|
|
||||||
_ => return Err(io::Error::last_os_error()),
|
|
||||||
}
|
|
||||||
let mut status = 0;
|
|
||||||
let rc = GetExitCodeProcess(child.as_raw_handle(), &mut status);
|
|
||||||
if rc == FALSE {
|
|
||||||
Err(io::Error::last_os_error())
|
|
||||||
} else {
|
|
||||||
Ok(Some(ExitStatus::from_raw(status)))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
pub(crate) type ChildStdin = PollEvented<NamedPipe>;
|
pub(crate) type ChildStdin = PollEvented<NamedPipe>;
|
||||||
pub(crate) type ChildStdout = PollEvented<NamedPipe>;
|
pub(crate) type ChildStdout = PollEvented<NamedPipe>;
|
||||||
pub(crate) type ChildStderr = PollEvented<NamedPipe>;
|
pub(crate) type ChildStderr = PollEvented<NamedPipe>;
|
||||||
|
@ -40,7 +40,7 @@ async fn issue_2174() {
|
|||||||
|
|
||||||
// Kill the child process.
|
// Kill the child process.
|
||||||
child.kill().unwrap();
|
child.kill().unwrap();
|
||||||
let _ = child.await;
|
let _ = child.wait().await;
|
||||||
|
|
||||||
assert_err!(handle.await);
|
assert_err!(handle.await);
|
||||||
}
|
}
|
||||||
|
@ -18,14 +18,16 @@ async fn issue_42() {
|
|||||||
let join_handles = (0..10usize).map(|_| {
|
let join_handles = (0..10usize).map(|_| {
|
||||||
task::spawn(async {
|
task::spawn(async {
|
||||||
let processes = (0..10usize).map(|i| {
|
let processes = (0..10usize).map(|i| {
|
||||||
Command::new("echo")
|
let mut child = Command::new("echo")
|
||||||
.arg(format!("I am spawned process #{}", i))
|
.arg(format!("I am spawned process #{}", i))
|
||||||
.stdin(Stdio::null())
|
.stdin(Stdio::null())
|
||||||
.stdout(Stdio::null())
|
.stdout(Stdio::null())
|
||||||
.stderr(Stdio::null())
|
.stderr(Stdio::null())
|
||||||
.kill_on_drop(true)
|
.kill_on_drop(true)
|
||||||
.spawn()
|
.spawn()
|
||||||
.unwrap()
|
.unwrap();
|
||||||
|
|
||||||
|
async move { child.wait().await }
|
||||||
});
|
});
|
||||||
|
|
||||||
join_all(processes).await;
|
join_all(processes).await;
|
||||||
|
@ -18,12 +18,17 @@ async fn simple() {
|
|||||||
|
|
||||||
let mut child = cmd.arg("exit 2").spawn().unwrap();
|
let mut child = cmd.arg("exit 2").spawn().unwrap();
|
||||||
|
|
||||||
let id = child.id();
|
let id = child.id().expect("missing id");
|
||||||
assert!(id > 0);
|
assert!(id > 0);
|
||||||
|
|
||||||
let status = assert_ok!((&mut child).await);
|
let status = assert_ok!(child.wait().await);
|
||||||
assert_eq!(status.code(), Some(2));
|
assert_eq!(status.code(), Some(2));
|
||||||
|
|
||||||
assert_eq!(child.id(), id);
|
// test that the `.wait()` method is fused just like the stdlib
|
||||||
|
let status = assert_ok!(child.wait().await);
|
||||||
|
assert_eq!(status.code(), Some(2));
|
||||||
|
|
||||||
|
// Can't get id after process has exited
|
||||||
|
assert_eq!(child.id(), None);
|
||||||
drop(child.kill());
|
drop(child.kill());
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user