mirror of
https://github.com/tokio-rs/tokio.git
synced 2025-09-25 12:00:35 +00:00
process: move into the tokio-net crate (#1475)
This commit is contained in:
parent
34a9dc2d76
commit
357df38861
@ -9,7 +9,6 @@ members = [
|
||||
"tokio-io",
|
||||
"tokio-macros",
|
||||
"tokio-net",
|
||||
"tokio-process",
|
||||
"tokio-sync",
|
||||
"tokio-test",
|
||||
"tokio-timer",
|
||||
|
@ -48,11 +48,11 @@ jobs:
|
||||
crates:
|
||||
tokio-fs: []
|
||||
tokio-net:
|
||||
- process
|
||||
- signal
|
||||
- tcp
|
||||
- udp
|
||||
- uds
|
||||
tokio-process: []
|
||||
|
||||
# Test crates that are NOT platform specific
|
||||
- template: ci/azure-test-stable.yml
|
||||
|
@ -22,12 +22,29 @@ categories = ["asynchronous", "network-programming"]
|
||||
|
||||
[features]
|
||||
async-traits = []
|
||||
process = [
|
||||
"crossbeam-queue",
|
||||
"futures-util-preview",
|
||||
"libc",
|
||||
"mio-named-pipes",
|
||||
"signal",
|
||||
"winapi/handleapi",
|
||||
"winapi/winerror",
|
||||
"winapi/minwindef",
|
||||
"winapi/processthreadsapi",
|
||||
"winapi/synchapi",
|
||||
"winapi/threadpoollegacyapiset",
|
||||
"winapi/winbase",
|
||||
"winapi/winnt",
|
||||
]
|
||||
signal = [
|
||||
"futures-util-preview",
|
||||
"mio-uds",
|
||||
"libc",
|
||||
"signal-hook-registry",
|
||||
"winapi",
|
||||
"winapi/consoleapi",
|
||||
"winapi/minwindef",
|
||||
"winapi/wincon"
|
||||
]
|
||||
tcp = [
|
||||
"bytes",
|
||||
@ -71,13 +88,17 @@ iovec = { version = "0.1", optional = true }
|
||||
|
||||
[target.'cfg(unix)'.dependencies]
|
||||
# UDS / Signal
|
||||
crossbeam-queue = { version = "0.1.2", optional = true }
|
||||
mio-uds = { version = "0.6.5", optional = true }
|
||||
libc = { version = "0.2.42", optional = true }
|
||||
signal-hook-registry = { version = "~1", optional = true }
|
||||
|
||||
[target.'cfg(windows)'.dependencies]
|
||||
mio-named-pipes = { version = "0.1", optional = true }
|
||||
|
||||
[target.'cfg(windows)'.dependencies.winapi]
|
||||
version = "0.3"
|
||||
features = ["consoleapi", "minwindef", "wincon"]
|
||||
default-features = false
|
||||
optional = true
|
||||
|
||||
[dev-dependencies]
|
||||
|
@ -1,5 +1,6 @@
|
||||
// A cat-like utility that can be used as a subprocess to test I/O
|
||||
// stream communication.
|
||||
//! A cat-like utility that can be used as a subprocess to test I/O
|
||||
//! stream communication.
|
||||
|
||||
use std::io;
|
||||
use std::io::Write;
|
||||
|
@ -40,6 +40,9 @@
|
||||
pub mod driver;
|
||||
pub mod util;
|
||||
|
||||
#[cfg(feature = "process")]
|
||||
pub mod process;
|
||||
|
||||
#[cfg(feature = "signal")]
|
||||
pub mod signal;
|
||||
|
||||
|
@ -1,16 +1,6 @@
|
||||
#![doc(html_root_url = "https://docs.rs/tokio-process/0.3.0-alpha.1")]
|
||||
#![warn(
|
||||
missing_debug_implementations,
|
||||
missing_docs,
|
||||
rust_2018_idioms,
|
||||
unreachable_pub
|
||||
)]
|
||||
#![doc(test(no_crate_inject, attr(deny(rust_2018_idioms))))]
|
||||
#![feature(async_await)]
|
||||
|
||||
//! An implementation of asynchronous process management for Tokio.
|
||||
//!
|
||||
//! This crate provides a [Command](Command) struct that imitates the interface of the
|
||||
//! This module provides a [`Command`](Command) struct that imitates the interface of the
|
||||
//! [`std::process::Command`] type in the standard library, but provides asynchronous versions of
|
||||
//! functions that create processes. These functions (`spawn`, `status`, `output` and their
|
||||
//! variants) return "future aware" types that interoperate with Tokio. The asynchronous process
|
||||
@ -24,7 +14,7 @@
|
||||
//! ```no_run
|
||||
//! #![feature(async_await)]
|
||||
//!
|
||||
//! use tokio_process::Command;
|
||||
//! use tokio_net::process::Command;
|
||||
//!
|
||||
//! #[tokio::main]
|
||||
//! async fn main() -> Result<(), Box<dyn std::error::Error>> {
|
||||
@ -49,7 +39,7 @@
|
||||
//! ```no_run
|
||||
//! #![feature(async_await)]
|
||||
//!
|
||||
//! use tokio_process::Command;
|
||||
//! use tokio_net::process::Command;
|
||||
//!
|
||||
//! #[tokio::main]
|
||||
//! async fn main() -> Result<(), Box<dyn std::error::Error>> {
|
||||
@ -74,7 +64,7 @@
|
||||
//! use futures_util::stream::StreamExt;
|
||||
//! use std::process::{Stdio};
|
||||
//! use tokio::codec::{FramedRead, LinesCodec};
|
||||
//! use tokio_process::Command;
|
||||
//! use tokio_net::process::Command;
|
||||
//!
|
||||
//! #[tokio::main]
|
||||
//! async fn main() -> Result<(), Box<dyn std::error::Error>> {
|
||||
@ -117,18 +107,11 @@
|
||||
//! While similar to the standard library, this crate's `Child` type differs
|
||||
//! importantly in the behavior of `drop`. In the standard library, a child
|
||||
//! process will continue running after the instance of [`std::process::Child`]
|
||||
//! is dropped. In this crate, however, because [`tokio_process::Child`](Child) is a
|
||||
//! is dropped. In this crate, however, because [`tokio_net::process::Child`](Child) is a
|
||||
//! future of the child's `ExitStatus`, a child process is terminated if
|
||||
//! `tokio_process::Child` is dropped. The behavior of the standard library can
|
||||
//! `tokio_net::process::Child` is dropped. The behavior of the standard library can
|
||||
//! be regained with the [`Child::forget`] method.
|
||||
|
||||
#[cfg(unix)]
|
||||
#[macro_use]
|
||||
extern crate lazy_static;
|
||||
#[cfg(unix)]
|
||||
#[macro_use]
|
||||
extern crate log;
|
||||
|
||||
use std::ffi::OsStr;
|
||||
use std::fmt;
|
||||
use std::future::Future;
|
||||
@ -139,12 +122,11 @@ use std::process::{Command as StdCommand, ExitStatus, Output, Stdio};
|
||||
use std::task::Context;
|
||||
use std::task::Poll;
|
||||
|
||||
use self::kill::Kill;
|
||||
use futures_core::TryFuture;
|
||||
use futures_util::future;
|
||||
use futures_util::future::FutureExt;
|
||||
use futures_util::try_future::TryFutureExt;
|
||||
|
||||
use kill::Kill;
|
||||
use tokio_io::{AsyncRead, AsyncReadExt, AsyncWrite};
|
||||
|
||||
#[path = "unix/mod.rs"]
|
||||
@ -168,7 +150,7 @@ pub struct Command {
|
||||
std: StdCommand,
|
||||
}
|
||||
|
||||
struct SpawnedChild {
|
||||
pub(crate) struct SpawnedChild {
|
||||
child: imp::Child,
|
||||
stdin: Option<imp::ChildStdin>,
|
||||
stdout: Option<imp::ChildStdout>,
|
||||
@ -200,7 +182,7 @@ impl Command {
|
||||
/// Basic usage:
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
/// let command = Command::new("sh");
|
||||
/// ```
|
||||
pub fn new<S: AsRef<OsStr>>(program: S) -> Command {
|
||||
@ -214,7 +196,7 @@ impl Command {
|
||||
/// Only one argument can be passed per use. So instead of:
|
||||
///
|
||||
/// ```no_run
|
||||
/// # tokio_process::Command::new("sh")
|
||||
/// # tokio_net::process::Command::new("sh")
|
||||
/// .arg("-C /path/to/repo")
|
||||
/// # ;
|
||||
/// ```
|
||||
@ -222,7 +204,7 @@ impl Command {
|
||||
/// usage would be:
|
||||
///
|
||||
/// ```no_run
|
||||
/// # tokio_process::Command::new("sh")
|
||||
/// # tokio_net::process::Command::new("sh")
|
||||
/// .arg("-C")
|
||||
/// .arg("/path/to/repo")
|
||||
/// # ;
|
||||
@ -237,7 +219,7 @@ impl Command {
|
||||
/// Basic usage:
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .arg("-l")
|
||||
@ -259,7 +241,7 @@ impl Command {
|
||||
/// Basic usage:
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .args(&["-l", "-a"]);
|
||||
@ -283,7 +265,7 @@ impl Command {
|
||||
/// Basic usage:
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .env("PATH", "/bin");
|
||||
@ -307,7 +289,7 @@ impl Command {
|
||||
/// use std::process::{Stdio};
|
||||
/// use std::env;
|
||||
/// use std::collections::HashMap;
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let filtered_env : HashMap<String, String> =
|
||||
/// env::vars().filter(|&(ref k, _)|
|
||||
@ -337,7 +319,7 @@ impl Command {
|
||||
/// Basic usage:
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .env_remove("PATH");
|
||||
@ -354,7 +336,7 @@ impl Command {
|
||||
/// Basic usage:
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .env_clear();
|
||||
@ -379,7 +361,7 @@ impl Command {
|
||||
/// Basic usage:
|
||||
///
|
||||
/// ```no_run
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .current_dir("/bin");
|
||||
@ -405,7 +387,7 @@ impl Command {
|
||||
///
|
||||
/// ```no_run
|
||||
/// use std::process::{Stdio};
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .stdin(Stdio::null());
|
||||
@ -429,7 +411,7 @@ impl Command {
|
||||
///
|
||||
/// ```no_run
|
||||
/// use std::process::{Stdio};
|
||||
/// use tokio_process::Command;;
|
||||
/// use tokio_net::process::Command;;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .stdout(Stdio::null());
|
||||
@ -453,7 +435,7 @@ impl Command {
|
||||
///
|
||||
/// ```no_run
|
||||
/// use std::process::{Stdio};
|
||||
/// use tokio_process::Command;;
|
||||
/// use tokio_net::process::Command;;
|
||||
///
|
||||
/// let command = Command::new("ls")
|
||||
/// .stderr(Stdio::null());
|
||||
@ -482,7 +464,7 @@ impl Command {
|
||||
///
|
||||
/// ```no_run
|
||||
/// #![feature(async_await)]
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// async fn run_ls() -> std::process::ExitStatus {
|
||||
/// Command::new("ls")
|
||||
@ -528,7 +510,7 @@ impl Command {
|
||||
///
|
||||
/// ```no_run
|
||||
/// #![feature(async_await)]
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// async fn run_ls() -> std::process::ExitStatus {
|
||||
/// Command::new("ls")
|
||||
@ -577,7 +559,7 @@ impl Command {
|
||||
///
|
||||
/// ```no_run
|
||||
/// #![feature(async_await)]
|
||||
/// use tokio_process::Command;
|
||||
/// use tokio_net::process::Command;
|
||||
///
|
||||
/// async fn run_ls() {
|
||||
/// let output: std::process::Output = Command::new("ls")
|
||||
@ -661,7 +643,7 @@ impl<T: TryFuture + Kill + Unpin> Future for ChildDropGuard<T> {
|
||||
/// like the OS-assigned identifier and the stdio streams.
|
||||
///
|
||||
/// > **Note**: The behavior of `drop` on a child in this crate is *different
|
||||
/// > than the behavior of the standard library*. If a `tokio_process::Child` is
|
||||
/// > than the behavior of the standard library*. If a `tokio_net::process::Child` is
|
||||
/// > dropped before the process finishes then the process will be terminated.
|
||||
/// > In the standard library, however, the process continues executing. This is
|
||||
/// > done because futures in general take `drop` as a sign of cancellation, and
|
||||
@ -773,7 +755,7 @@ impl Child {
|
||||
///
|
||||
/// ```no_run
|
||||
/// # #![feature(async_await)]
|
||||
/// # use tokio_process::Command;
|
||||
/// # use tokio_net::process::Command;
|
||||
///
|
||||
/// # #[tokio::main]
|
||||
/// # async fn main() {
|
||||
@ -993,8 +975,7 @@ mod test {
|
||||
|
||||
use futures_util::future::FutureExt;
|
||||
|
||||
use crate::kill::Kill;
|
||||
|
||||
use super::kill::Kill;
|
||||
use super::ChildDropGuard;
|
||||
|
||||
struct Mock {
|
@ -27,11 +27,9 @@ mod reap;
|
||||
use self::orphan::{AtomicOrphanQueue, OrphanQueue, Wait};
|
||||
use self::reap::Reaper;
|
||||
use super::SpawnedChild;
|
||||
use crate::kill::Kill;
|
||||
|
||||
use tokio_net::signal::unix::{signal, Signal, SignalKind};
|
||||
use tokio_net::util::PollEvented;
|
||||
|
||||
use crate::process::kill::Kill;
|
||||
use crate::signal::unix::{signal, Signal, SignalKind};
|
||||
use crate::util::PollEvented;
|
||||
use mio::event::Evented;
|
||||
use mio::unix::{EventedFd, UnixReady};
|
||||
use mio::{Poll as MioPoll, PollOpt, Ready, Token};
|
||||
@ -60,7 +58,7 @@ impl Kill for process::Child {
|
||||
}
|
||||
}
|
||||
|
||||
lazy_static! {
|
||||
lazy_static::lazy_static! {
|
||||
static ref ORPHAN_QUEUE: AtomicOrphanQueue<process::Child> = AtomicOrphanQueue::new();
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
use crossbeam_queue::SegQueue;
|
||||
use log::error;
|
||||
use std::io;
|
||||
use std::process::ExitStatus;
|
||||
|
@ -1,5 +1,5 @@
|
||||
use super::orphan::{OrphanQueue, Wait};
|
||||
use crate::kill::Kill;
|
||||
use crate::process::kill::Kill;
|
||||
use futures_core::stream::Stream;
|
||||
use std::future::Future;
|
||||
use std::io;
|
@ -16,11 +16,8 @@
|
||||
//! from then on out.
|
||||
|
||||
use super::SpawnedChild;
|
||||
use crate::kill::Kill;
|
||||
|
||||
use tokio_net::util::PollEvented;
|
||||
use tokio_sync::oneshot;
|
||||
|
||||
use crate::process::kill::Kill;
|
||||
use crate::util::PollEvented;
|
||||
use futures_util::future::Fuse;
|
||||
use futures_util::future::FutureExt;
|
||||
use mio_named_pipes::NamedPipe;
|
||||
@ -30,22 +27,23 @@ use std::io;
|
||||
use std::os::windows::prelude::*;
|
||||
use std::os::windows::process::ExitStatusExt;
|
||||
use std::pin::Pin;
|
||||
use std::process::{self, ExitStatus};
|
||||
use std::process::{Child as StdChild, Command as StdCommand, ExitStatus};
|
||||
use std::ptr;
|
||||
use std::task::Context;
|
||||
use std::task::Poll;
|
||||
use winapi::shared::minwindef::*;
|
||||
use winapi::shared::winerror::*;
|
||||
use winapi::um::handleapi::*;
|
||||
use winapi::um::processthreadsapi::*;
|
||||
use winapi::um::synchapi::*;
|
||||
use winapi::um::threadpoollegacyapiset::*;
|
||||
use winapi::um::winbase::*;
|
||||
use winapi::um::winnt::*;
|
||||
use tokio_sync::oneshot;
|
||||
use winapi::shared::minwindef::FALSE;
|
||||
use winapi::shared::winerror::WAIT_TIMEOUT;
|
||||
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::winbase::{RegisterWaitForSingleObject, INFINITE, WAIT_OBJECT_0};
|
||||
use winapi::um::winnt::{BOOLEAN, HANDLE, PVOID, WT_EXECUTEINWAITTHREAD, WT_EXECUTEONLYONCE};
|
||||
|
||||
#[must_use = "futures do nothing unless polled"]
|
||||
pub(crate) struct Child {
|
||||
child: process::Child,
|
||||
child: StdChild,
|
||||
waiting: Option<Waiting>,
|
||||
}
|
||||
|
||||
@ -68,7 +66,7 @@ struct Waiting {
|
||||
unsafe impl Sync for Waiting {}
|
||||
unsafe impl Send for Waiting {}
|
||||
|
||||
pub(crate) fn spawn_child(cmd: &mut process::Command) -> io::Result<SpawnedChild> {
|
||||
pub(crate) fn spawn_child(cmd: &mut StdCommand) -> io::Result<SpawnedChild> {
|
||||
let mut child = cmd.spawn()?;
|
||||
let stdin = stdio(child.stdin.take());
|
||||
let stdout = stdio(child.stdout.take());
|
||||
@ -160,7 +158,7 @@ unsafe extern "system" fn callback(ptr: PVOID, _timer_fired: BOOLEAN) {
|
||||
let _ = complete.take().unwrap().send(());
|
||||
}
|
||||
|
||||
pub(crate) fn try_wait(child: &process::Child) -> io::Result<Option<ExitStatus>> {
|
||||
pub(crate) fn try_wait(child: &StdChild) -> io::Result<Option<ExitStatus>> {
|
||||
unsafe {
|
||||
match WaitForSingleObject(child.as_raw_handle(), 0) {
|
||||
WAIT_OBJECT_0 => {}
|
@ -1,9 +1,8 @@
|
||||
//! Asynchronous signal handling for Tokio
|
||||
//!
|
||||
//! This crate implements asynchronous signal handling for Tokio, an
|
||||
//! asynchronous I/O framework in Rust. The primary type exported from this
|
||||
//! crate, `unix::Signal`, allows listening for arbitrary signals on Unix
|
||||
//! platforms, receiving them in an asynchronous fashion.
|
||||
//! The primary type exported from this crate, `unix::Signal`, allows
|
||||
//! listening for arbitrary signals on Unix platforms, receiving them
|
||||
//! in an asynchronous fashion.
|
||||
//!
|
||||
//! Note that signal handling is in general a very tricky topic and should be
|
||||
//! used with great care. This crate attempts to implement 'best practice' for
|
||||
|
@ -1,3 +1,4 @@
|
||||
#![cfg(feature = "process")]
|
||||
#![cfg(unix)]
|
||||
#![warn(rust_2018_idioms)]
|
||||
|
||||
@ -10,9 +11,10 @@ use std::sync::Arc;
|
||||
use std::thread;
|
||||
use std::time::Duration;
|
||||
use tokio::runtime::current_thread;
|
||||
use tokio_process::Command;
|
||||
use tokio_net::process::Command;
|
||||
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
fn run_test() {
|
||||
let finished = Arc::new(AtomicBool::new(false));
|
||||
@ -34,7 +36,7 @@ fn run_test() {
|
||||
}
|
||||
|
||||
let mut rt = current_thread::Runtime::new().expect("failed to get runtime");
|
||||
rt.block_on(support::with_timeout(futures.collect::<Vec<_>>()));
|
||||
rt.block_on(with_timeout(futures.collect::<Vec<_>>()));
|
||||
drop(rt);
|
||||
|
||||
finished_clone.store(true, Ordering::SeqCst);
|
34
tokio-net/tests/process_smoke.rs
Normal file
34
tokio-net/tests/process_smoke.rs
Normal file
@ -0,0 +1,34 @@
|
||||
#![cfg(feature = "process")]
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
use tokio_net::process::Command;
|
||||
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn simple() {
|
||||
let mut cmd;
|
||||
|
||||
if cfg!(windows) {
|
||||
cmd = Command::new("cmd");
|
||||
cmd.arg("/c");
|
||||
} else {
|
||||
cmd = Command::new("sh");
|
||||
cmd.arg("-c");
|
||||
}
|
||||
|
||||
let mut child = cmd.arg("exit 2").spawn().unwrap();
|
||||
|
||||
let id = child.id();
|
||||
assert!(id > 0);
|
||||
|
||||
let status = with_timeout(&mut child)
|
||||
.await
|
||||
.expect("failed to run future");
|
||||
assert_eq!(status.code(), Some(2));
|
||||
|
||||
assert_eq!(child.id(), id);
|
||||
drop(child.kill());
|
||||
}
|
@ -1,9 +1,11 @@
|
||||
#![cfg(feature = "process")]
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
#[macro_use]
|
||||
extern crate log;
|
||||
|
||||
use std::env;
|
||||
use std::io;
|
||||
use std::process::{ExitStatus, Stdio};
|
||||
|
||||
@ -12,12 +14,22 @@ use futures_util::future::FutureExt;
|
||||
use futures_util::stream::StreamExt;
|
||||
use tokio::codec::{FramedRead, LinesCodec};
|
||||
use tokio::io::AsyncWriteExt;
|
||||
use tokio_process::{Child, Command};
|
||||
use tokio_net::process::{Child, Command};
|
||||
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
fn cat() -> Command {
|
||||
let mut cmd = support::cmd("cat");
|
||||
let mut me = env::current_exe().unwrap();
|
||||
me.pop();
|
||||
|
||||
if me.ends_with("deps") {
|
||||
me.pop();
|
||||
}
|
||||
|
||||
me.push("test-cat");
|
||||
|
||||
let mut cmd = Command::new(me);
|
||||
cmd.stdin(Stdio::piped()).stdout(Stdio::piped());
|
||||
cmd
|
||||
}
|
||||
@ -98,7 +110,7 @@ async fn feed_cat(mut cat: Child, n: usize) -> io::Result<ExitStatus> {
|
||||
#[tokio::test]
|
||||
async fn feed_a_lot() {
|
||||
let child = cat().spawn().unwrap();
|
||||
let status = support::with_timeout(feed_cat(child, 10000)).await.unwrap();
|
||||
let status = with_timeout(feed_cat(child, 10000)).await.unwrap();
|
||||
assert_eq!(status.code(), Some(0));
|
||||
}
|
||||
|
||||
@ -116,7 +128,7 @@ async fn wait_with_output_captures() {
|
||||
out.await
|
||||
};
|
||||
|
||||
let output = support::with_timeout(future).await.unwrap();
|
||||
let output = with_timeout(future).await.unwrap();
|
||||
|
||||
assert!(output.status.success());
|
||||
assert_eq!(output.stdout, write_bytes);
|
||||
@ -130,7 +142,7 @@ async fn status_closes_any_pipes() {
|
||||
// we would end up blocking forever (and time out).
|
||||
let child = cat().status().expect("failed to spawn child");
|
||||
|
||||
support::with_timeout(child)
|
||||
with_timeout(child)
|
||||
.await
|
||||
.expect("time out exceeded! did we get stuck waiting on the child?");
|
||||
}
|
@ -2,8 +2,8 @@
|
||||
#![cfg(feature = "signal")]
|
||||
#![warn(rust_2018_idioms)]
|
||||
|
||||
pub mod signal_support;
|
||||
use crate::signal_support::*;
|
||||
pub mod support;
|
||||
use support::*;
|
||||
|
||||
#[test]
|
||||
fn dropping_loops_does_not_cause_starvation() {
|
||||
|
@ -3,8 +3,8 @@
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
pub mod signal_support;
|
||||
use crate::signal_support::*;
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn drop_then_get_a_signal() {
|
||||
|
@ -3,8 +3,8 @@
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
pub mod signal_support;
|
||||
use crate::signal_support::*;
|
||||
pub mod support;
|
||||
use support::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn dropping_signal_does_not_deregister_any_other_instances() {
|
||||
|
@ -2,8 +2,8 @@
|
||||
#![cfg(feature = "signal")]
|
||||
#![warn(rust_2018_idioms)]
|
||||
|
||||
pub mod signal_support;
|
||||
use crate::signal_support::*;
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
use std::sync::mpsc::channel;
|
||||
use std::thread;
|
||||
|
@ -3,8 +3,8 @@
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
pub mod signal_support;
|
||||
use crate::signal_support::*;
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn notify_both() {
|
||||
|
@ -3,8 +3,8 @@
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
pub mod signal_support;
|
||||
use crate::signal_support::*;
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn simple() {
|
||||
|
@ -3,8 +3,8 @@
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
pub mod signal_support;
|
||||
use crate::signal_support::*;
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn twice() {
|
||||
|
@ -1,19 +1,22 @@
|
||||
#![cfg(unix)]
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![allow(dead_code)]
|
||||
|
||||
pub use tokio::runtime::current_thread::{self, Runtime as CurrentThreadRuntime};
|
||||
use tokio::timer::Timeout;
|
||||
|
||||
#[cfg(all(unix, feature = "signal"))]
|
||||
pub use tokio_net::signal::unix::{signal, SignalKind};
|
||||
|
||||
pub use futures_util::future;
|
||||
use futures_util::future::FutureExt;
|
||||
pub use futures_util::stream::StreamExt;
|
||||
#[cfg(unix)]
|
||||
use libc::{c_int, getpid, kill};
|
||||
use std::future::Future;
|
||||
use std::time::Duration;
|
||||
|
||||
pub fn with_timeout<F: Future>(future: F) -> impl Future<Output = F::Output> {
|
||||
Timeout::new(future, Duration::from_secs(1)).map(Result::unwrap)
|
||||
Timeout::new(future, Duration::from_secs(3)).map(Result::unwrap)
|
||||
}
|
||||
|
||||
pub fn run_with_timeout<F>(rt: &mut CurrentThreadRuntime, future: F) -> F::Output
|
@ -1,76 +0,0 @@
|
||||
# 0.3.0-alpha.1 (August 8, 2019)
|
||||
|
||||
### Changed
|
||||
- Switch to `async`, `await`, and `std::future`.
|
||||
|
||||
## 0.2.4 - 2019-06-21
|
||||
### Fixed
|
||||
* Proccesses "leaked" via `Child::forget` now reaped rather than left as zombies
|
||||
for the duration of the parent process.
|
||||
* Dropping a `Child` process no longer blocks the caller until the process fully
|
||||
exits. This avoids a pathological deadlock if the kernel doesn't kill the child.
|
||||
|
||||
### Changed
|
||||
* Updated the example program for reading lines from a child process to be more
|
||||
flexible to be copy/pasted and iterated upon.
|
||||
|
||||
## 0.2.3 - 2018-11-01
|
||||
### Added
|
||||
* `ChildStd{in, out, err}` now implement `AsRawFd`/`AsRawHandle` on Unix/Windows
|
||||
systems, respectively.
|
||||
|
||||
## 0.2.2 - 2018-05-27
|
||||
### Fixed
|
||||
- Fixed a pathological situation where a signal could be missed if it arrived
|
||||
after polling the child but before registering for a new notification
|
||||
|
||||
## 0.2.1 - 2018-05-18
|
||||
### Changed
|
||||
- **Breaking**: asynchronous spawning of a child process now requires using a
|
||||
reactor handle from the `tokio` crate instead of the `tokio-core` crate
|
||||
- Child processes may be spawned without specifying a `tokio` handle at all
|
||||
(the current/default reactor handle will be used)
|
||||
### Removed
|
||||
- **Breaking**: removed all previously deprecated items
|
||||
|
||||
## 0.1.6 - 2018-05-09
|
||||
### Fixed
|
||||
- On Unix systems, any child processes that are `kill`ed (or implicitly killed
|
||||
via dropping the child without calling `forget`) are no longer left in a zombie
|
||||
state, which allows the OS to reclaim the process.
|
||||
|
||||
## 0.1.5 - 2018-01-03
|
||||
### Changed
|
||||
- Minimum required version of `winapi` has been bumped to `0.3`.
|
||||
|
||||
## 0.1.4 - 2017-06-25
|
||||
### Fixed
|
||||
- Added missing `Debug` impls on all types.
|
||||
- Added missing `must_use` annotations on all futures.
|
||||
- Ensure `status_async` closes child's stdio handles after spawning in order
|
||||
to prevent potential deadlocks when attempting to interact with any pipes held
|
||||
by the parent process.
|
||||
|
||||
## 0.1.3 - 2017-03-15
|
||||
### Changed
|
||||
- Minimum required version of `futures` has been bumped to `0.1.11`.
|
||||
- Minimum required version of `mio` has been bumped to `0.6.5`.
|
||||
- Minimum required version of `tokio-core` has been bumped to `0.1.6`.
|
||||
|
||||
## 0.1.2 - 2017-01-24
|
||||
### Changed
|
||||
- Minimum required version of `tokio-signal` has been bumped to `0.1.2`.
|
||||
### Fixed
|
||||
- The event loop which spawns the first async child no longer needs to be kept
|
||||
alive for subsequent child spawns to make progress.
|
||||
|
||||
## 0.1.1 - 2016-12-19
|
||||
### Added
|
||||
- Support performing async I/O operations on the child's stdio handles.
|
||||
### Changed
|
||||
- Functionality has been reimplemented as the `CommandExt` extension trait
|
||||
(implemented directly on `std::process::Command`) instead of going through
|
||||
the locally vendored `Command` type.
|
||||
|
||||
## 0.1.0 - 2016-09-10
|
||||
- First release!
|
@ -1,61 +0,0 @@
|
||||
[package]
|
||||
name = "tokio-process"
|
||||
# When releasing to crates.io:
|
||||
# - Remove path dependencies
|
||||
# - Update html_root_url.
|
||||
# - Update doc url
|
||||
# - Cargo.toml
|
||||
# - Update CHANGELOG.md.
|
||||
# - Create "v0.1.x" git tag.
|
||||
version = "0.3.0-alpha.1"
|
||||
edition = "2018"
|
||||
authors = ["Tokio Contributors <team@tokio.rs>"]
|
||||
license = "MIT"
|
||||
repository = "https://github.com/tokio-rs/tokio"
|
||||
homepage = "https://github.com/tokio-rs/tokio"
|
||||
documentation = "https://docs.rs/tokio-process/0.2.4/tokio_process"
|
||||
description = """
|
||||
An implementation of an asynchronous process management backed futures.
|
||||
"""
|
||||
categories = ["asynchronous"]
|
||||
|
||||
[dependencies]
|
||||
futures-core-preview = "=0.3.0-alpha.18"
|
||||
futures-util-preview = "=0.3.0-alpha.18"
|
||||
log = "0.4"
|
||||
tokio-io = { version = "=0.2.0-alpha.2", path = "../tokio-io", features = ["util"] }
|
||||
tokio-net = { version = "=0.2.0-alpha.2", path = "../tokio-net" }
|
||||
|
||||
[dev-dependencies.tokio]
|
||||
version = "0.2.0-alpha.1"
|
||||
path = "../tokio"
|
||||
default-features = false
|
||||
features = ["codec", "rt-full"]
|
||||
|
||||
[target.'cfg(windows)'.dependencies]
|
||||
mio-named-pipes = "0.1"
|
||||
tokio-sync = { version = "=0.2.0-alpha.2", path = "../tokio-sync" }
|
||||
|
||||
[target.'cfg(windows)'.dependencies.winapi]
|
||||
version = "0.3"
|
||||
features = [
|
||||
"handleapi",
|
||||
"winerror",
|
||||
"minwindef",
|
||||
"processthreadsapi",
|
||||
"synchapi",
|
||||
"threadpoollegacyapiset",
|
||||
"winbase",
|
||||
"winnt",
|
||||
]
|
||||
|
||||
[target.'cfg(unix)'.dependencies]
|
||||
crossbeam-queue = "0.1.2"
|
||||
lazy_static = "1.3"
|
||||
libc = "0.2"
|
||||
log = "0.4"
|
||||
mio = "0.6.5"
|
||||
tokio-net = { version = "=0.2.0-alpha.2", path = "../tokio-net", features = ["signal"] }
|
||||
|
||||
[package.metadata.docs.rs]
|
||||
all-features = true
|
@ -1,25 +0,0 @@
|
||||
Copyright (c) 2019 Tokio Contributors
|
||||
|
||||
Permission is hereby granted, free of charge, to any
|
||||
person obtaining a copy of this software and associated
|
||||
documentation files (the "Software"), to deal in the
|
||||
Software without restriction, including without
|
||||
limitation the rights to use, copy, modify, merge,
|
||||
publish, distribute, sublicense, and/or sell copies of
|
||||
the Software, and to permit persons to whom the Software
|
||||
is furnished to do so, subject to the following
|
||||
conditions:
|
||||
|
||||
The above copyright notice and this permission notice
|
||||
shall be included in all copies or substantial portions
|
||||
of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF
|
||||
ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED
|
||||
TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A
|
||||
PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT
|
||||
SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
|
||||
CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
|
||||
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR
|
||||
IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
DEALINGS IN THE SOFTWARE.
|
@ -1,13 +0,0 @@
|
||||
# tokio-process
|
||||
|
||||
An implementation of process management for Tokio
|
||||
|
||||
## License
|
||||
|
||||
This project is licensed under the [MIT license](./LICENSE).
|
||||
|
||||
### Contribution
|
||||
|
||||
Unless you explicitly state otherwise, any contribution intentionally submitted
|
||||
for inclusion in Tokio by you, shall be licensed as MIT, without any additional
|
||||
terms or conditions.
|
@ -1,5 +0,0 @@
|
||||
#[allow(dead_code)]
|
||||
|
||||
fn main() {
|
||||
std::process::exit(std::env::args().nth(1).unwrap().parse().unwrap());
|
||||
}
|
@ -1,23 +0,0 @@
|
||||
#![warn(rust_2018_idioms)]
|
||||
#![feature(async_await)]
|
||||
|
||||
mod support;
|
||||
|
||||
#[tokio::test]
|
||||
async fn simple() {
|
||||
let mut cmd = support::cmd("exit");
|
||||
cmd.arg("2");
|
||||
|
||||
let mut child = cmd.spawn().unwrap();
|
||||
|
||||
let id = child.id();
|
||||
assert!(id > 0);
|
||||
|
||||
let status = support::with_timeout(&mut child)
|
||||
.await
|
||||
.expect("failed to run future");
|
||||
assert_eq!(status.code(), Some(2));
|
||||
|
||||
assert_eq!(child.id(), id);
|
||||
drop(child.kill());
|
||||
}
|
@ -1,29 +0,0 @@
|
||||
#![warn(rust_2018_idioms)]
|
||||
|
||||
use futures_util::future;
|
||||
use futures_util::future::FutureExt;
|
||||
use std::env;
|
||||
use std::future::Future;
|
||||
use std::time::Duration;
|
||||
use tokio::timer::Timeout;
|
||||
use tokio_process::Command;
|
||||
|
||||
#[allow(dead_code)]
|
||||
pub fn cmd(s: &str) -> Command {
|
||||
let mut me = env::current_exe().unwrap();
|
||||
me.pop();
|
||||
if me.ends_with("deps") {
|
||||
me.pop();
|
||||
}
|
||||
me.push(s);
|
||||
Command::new(me)
|
||||
}
|
||||
|
||||
pub fn with_timeout<F: Future>(future: F) -> impl Future<Output = F::Output> {
|
||||
Timeout::new(future, Duration::from_secs(3)).then(|r| {
|
||||
if r.is_err() {
|
||||
panic!("timed out {:?}", r.err());
|
||||
}
|
||||
future::ready(r.unwrap())
|
||||
})
|
||||
}
|
Loading…
x
Reference in New Issue
Block a user