feat(lib): update Tokio, bytes, http, h2, and http-body

This commit is contained in:
Sean McArthur
2019-12-03 14:36:20 -08:00
parent 131962c86a
commit cb3f39c2dc
51 changed files with 985 additions and 1305 deletions

View File

@@ -20,35 +20,28 @@ include = [
]
[dependencies]
bytes = "0.4.6"
futures-core = "0.3.1"
futures-channel = "0.3.1"
futures-util = "0.3.1"
http = "0.1.15"
http-body = "=0.2.0-alpha.3"
bytes = "0.5"
futures-core = { version = "0.3", default-features = false }
futures-channel = "0.3"
futures-util = { version = "0.3", default-features = false }
http = "0.2"
http-body = "0.2"
httparse = "1.0"
h2 = "=0.2.0-alpha.3"
iovec = "0.1"
h2 = "0.2"
itoa = "0.4.1"
log = "0.4"
pin-project = "0.4"
time = "0.1"
tower-service = "=0.3.0-alpha.2"
tokio-executor = "=0.2.0-alpha.6"
tokio-io = "=0.2.0-alpha.6"
tokio-sync = "=0.2.0-alpha.6"
tokio = { version = "0.2", features = ["sync"] }
want = "0.3"
# Optional
net2 = { version = "0.2.32", optional = true }
tokio = { version = "=0.2.0-alpha.6", optional = true, default-features = false, features = ["rt-full"] }
tokio-net = { version = "=0.2.0-alpha.6", optional = true, features = ["tcp"] }
tokio-timer = { version = "=0.3.0-alpha.6", optional = true }
[dev-dependencies]
futures-util-a19 = { version = "=0.3.0-alpha.19", package = "futures-util-preview" }
futures-util = { version = "0.3", default-features = false, features = ["alloc"] }
matches = "0.1"
num_cpus = "1.0"
pretty_env_logger = "0.3"
@@ -56,9 +49,8 @@ spmc = "0.3"
serde = "1.0"
serde_derive = "1.0"
serde_json = "1.0"
tokio = "=0.2.0-alpha.6" # using #[tokio::test] attributes
tokio-fs = "=0.2.0-alpha.6"
tokio-test = "=0.2.0-alpha.6"
tokio = { version = "0.2.2", features = ["fs", "macros", "rt-util", "sync", "time", "test-util"] }
tokio-test = "0.2"
url = "1.0"
[features]
@@ -68,13 +60,13 @@ default = [
]
runtime = [
"tcp",
"tokio",
"tokio/time",
]
tcp = [
"net2",
"tokio-executor/blocking",
"tokio-net",
"tokio-timer",
"tokio/blocking",
"tokio/tcp",
"tokio/time",
]
# unstable features
@@ -206,3 +198,4 @@ required-features = ["runtime", "unstable-stream"]
name = "server"
path = "tests/server.rs"
required-features = ["runtime"]

View File

@@ -3,8 +3,8 @@
extern crate test;
use std::net::SocketAddr;
use tokio::net::TcpListener;
use tokio::runtime::current_thread::Runtime;
use hyper::client::connect::{Destination, HttpConnector};
use hyper::service::Service;
use http::Uri;
@@ -12,8 +12,12 @@ use http::Uri;
#[bench]
fn http_connector(b: &mut test::Bencher) {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let mut listener = rt.block_on(TcpListener::bind("127.0.0.1:0")).expect("bind");
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.expect("rt build");
let mut listener = rt.block_on(TcpListener::bind(&SocketAddr::from(([127, 0, 0, 1], 0)))).expect("bind");
let addr = listener.local_addr().expect("local_addr");
let uri: Uri = format!("http://{}/", addr).parse().expect("uri parse");
let dst = Destination::try_from_uri(uri).expect("destination");

View File

@@ -6,9 +6,8 @@ extern crate test;
use std::net::SocketAddr;
use futures_util::future::join_all;
use tokio::runtime::current_thread::Runtime;
use hyper::{Body, Method, Request, Response, Server};
use hyper::{body::HttpBody as _, Body, Method, Request, Response, Server};
use hyper::client::HttpConnector;
// HTTP1
@@ -264,8 +263,12 @@ impl Opts {
fn bench(self, b: &mut test::Bencher) {
let _ = pretty_env_logger::try_init();
// Create a runtime of current thread.
let mut rt = Runtime::new().unwrap();
let exec = rt.handle();
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.expect("rt build");
let exec = rt.handle().clone();
let req_len = self.request_body.map(|b| b.len()).unwrap_or(0) as u64;
let req_len = if self.request_chunks > 0 {
@@ -297,7 +300,7 @@ impl Opts {
for _ in 0..chunk_cnt {
tx.send_data(chunk.into()).await.expect("send_data");
}
}).expect("body tx spawn");
});
body
} else {
self
@@ -340,12 +343,13 @@ impl Opts {
}
}
fn spawn_server(rt: &mut Runtime, opts: &Opts) -> SocketAddr {
fn spawn_server(rt: &mut tokio::runtime::Runtime, opts: &Opts) -> SocketAddr {
use hyper::service::{make_service_fn, service_fn};
let addr = "127.0.0.1:0".parse().unwrap();
let body = opts.response_body;
let srv = Server::bind(&addr)
let srv = rt.block_on(async move {
Server::bind(&addr)
.http2_only(opts.http2)
.http2_initial_stream_window_size(opts.http2_stream_window)
.http2_initial_connection_window_size(opts.http2_conn_window)
@@ -355,7 +359,8 @@ fn spawn_server(rt: &mut Runtime, opts: &Opts) -> SocketAddr {
while let Some(_chunk) = req_body.next().await {}
Ok::<_, hyper::Error>(Response::new(Body::from(body)))
}))
}));
}))
});
let addr = srv.local_addr();
rt.spawn(async {
if let Err(err) = srv.await {

View File

@@ -8,7 +8,6 @@ use std::net::{TcpStream};
use std::sync::mpsc;
use std::time::Duration;
use tokio::runtime::current_thread;
use tokio::sync::oneshot;
use hyper::{Body, Response, Server};
@@ -31,9 +30,17 @@ fn hello_world(b: &mut test::Bencher) {
Ok::<_, hyper::Error>(Response::new(Body::from("Hello, World!")))
}))
});
let srv = Server::bind(&addr)
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.expect("rt build");
let srv = rt.block_on(async move {
Server::bind(&addr)
.http1_pipeline_flush(true)
.serve(make_svc);
.serve(make_svc)
});
addr_tx.send(srv.local_addr()).unwrap();
@@ -42,13 +49,11 @@ fn hello_world(b: &mut test::Bencher) {
until_rx.await.ok();
});
let mut rt = current_thread::Runtime::new().unwrap();
rt.spawn(async {
rt.block_on(async {
if let Err(e) = graceful.await {
panic!("server error: {}", e);
}
});
rt.run().unwrap();
});
addr_rx.recv().unwrap()

View File

@@ -9,7 +9,6 @@ use std::sync::mpsc;
use std::time::Duration;
use futures_util::{stream, StreamExt};
use tokio::runtime::current_thread;
use tokio::sync::oneshot;
use hyper::{Body, Response, Server};
@@ -33,8 +32,17 @@ macro_rules! bench_server {
)
}))
});
let srv = Server::bind(&addr)
.serve(make_svc);
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.expect("rt build");
let srv = rt.block_on(async move {
Server::bind(&addr)
.serve(make_svc)
});
addr_tx.send(srv.local_addr()).unwrap();
@@ -42,13 +50,11 @@ macro_rules! bench_server {
.with_graceful_shutdown(async {
until_rx.await.ok();
});
let mut rt = current_thread::Runtime::new().unwrap();
rt.spawn(async {
rt.block_on(async move {
if let Err(e) = graceful.await {
panic!("server error: {}", e);
}
});
rt.run().unwrap();
});
addr_rx.recv().unwrap()

View File

@@ -4,6 +4,7 @@ use std::env;
use std::io::{self, Write};
use hyper::Client;
use futures_util::StreamExt;
// A simple type alias so as to DRY.
type Result<T> = std::result::Result<T, Box<dyn std::error::Error + Send + Sync>>;
@@ -24,7 +25,7 @@ async fn main() -> Result<()> {
// HTTPS requires picking a TLS implementation, so give a better
// warning if the user tries to request an 'https' URL.
let url = url.parse::<hyper::Uri>().unwrap();
if url.scheme_part().map(|s| s.as_ref()) != Some("http") {
if url.scheme_str() != Some("http") {
println!("This example only works with 'http' URLs.");
return Ok(());
}

View File

@@ -5,7 +5,7 @@
extern crate serde_derive;
use hyper::Client;
use futures_util::TryStreamExt;
use futures_util::StreamExt;
// A simple type alias so as to DRY.
type Result<T> = std::result::Result<T, Box<dyn std::error::Error + Send + Sync>>;
@@ -27,9 +27,12 @@ async fn fetch_json(url: hyper::Uri) -> Result<Vec<User>> {
let client = Client::new();
// Fetch the url...
let res = client.get(url).await?;
let mut res = client.get(url).await?;
// asynchronously concatenate chunks of the body
let body = res.into_body().try_concat().await?;
let mut body = Vec::new();
while let Some(chunk) = res.body_mut().next().await {
body.extend_from_slice(&chunk?);
}
// try to parse as json with serde_json
let users = serde_json::from_slice(&body)?;

View File

@@ -2,12 +2,11 @@
use hyper::{Body, Method, Request, Response, Server, StatusCode};
use hyper::service::{make_service_fn, service_fn};
use futures_util::TryStreamExt;
use futures_util::{StreamExt, TryStreamExt};
/// This is our service handler. It receives a Request, routes on its
/// path, and returns a Future of a Response.
async fn echo(req: Request<Body>) -> Result<Response<Body>, hyper::Error> {
async fn echo(mut req: Request<Body>) -> Result<Response<Body>, hyper::Error> {
match (req.method(), req.uri().path()) {
// Serve some instructions at /
(&Method::GET, "/") => {
@@ -37,13 +36,17 @@ async fn echo(req: Request<Body>) -> Result<Response<Body>, hyper::Error> {
// So here we do `.await` on the future, waiting on concatenating the full body,
// then afterwards the content can be reversed. Only then can we return a `Response`.
(&Method::POST, "/echo/reversed") => {
let whole_chunk = req.into_body().try_concat().await;
let mut whole_body = Vec::new();
while let Some(chunk) = req.body_mut().next().await {
whole_body.extend_from_slice(&chunk?);
}
let reversed_chunk = whole_chunk.map(move |chunk| {
chunk.iter().rev().cloned().collect::<Vec<u8>>()
})?;
Ok(Response::new(Body::from(reversed_chunk)))
let reversed_body = whole_body
.iter()
.rev()
.cloned()
.collect::<Vec<u8>>();
Ok(Response::new(Body::from(reversed_body)))
}
// Return the 404 Not Found for other routes.

View File

@@ -6,20 +6,24 @@ use hyper::service::{service_fn, make_service_fn};
use std::collections::HashMap;
use url::form_urlencoded;
use futures_util::TryStreamExt;
use futures_util::StreamExt;
static INDEX: &[u8] = b"<html><body><form action=\"post\" method=\"post\">Name: <input type=\"text\" name=\"name\"><br>Number: <input type=\"text\" name=\"number\"><br><input type=\"submit\"></body></html>";
static MISSING: &[u8] = b"Missing field";
static NOTNUMERIC: &[u8] = b"Number field is not numeric";
// Using service_fn, we can turn this function into a `Service`.
async fn param_example(req: Request<Body>) -> Result<Response<Body>, hyper::Error> {
async fn param_example(mut req: Request<Body>) -> Result<Response<Body>, hyper::Error> {
match (req.method(), req.uri().path()) {
(&Method::GET, "/") | (&Method::GET, "/post") => {
Ok(Response::new(INDEX.into()))
},
(&Method::POST, "/post") => {
let b = req.into_body().try_concat().await?;
// Concatenate the body...
let mut b = Vec::new();
while let Some(chunk) = req.body_mut().next().await {
b.extend_from_slice(&chunk?);
}
// Parse the request body. form_urlencoded::parse
// always succeeds, but in general parsing may
// fail (for example, an invalid post of json), so

View File

@@ -1,7 +1,7 @@
#![deny(warnings)]
use tokio::io::AsyncReadExt;
use tokio_fs::File;
use tokio::fs::File;
use hyper::{Body, Method, Result, Request, Response, Server, StatusCode};
use hyper::service::{make_service_fn, service_fn};

View File

@@ -1,3 +1,5 @@
fn main() {}
/*
#![deny(warnings)]
use std::cell::Cell;
@@ -5,14 +7,24 @@ use std::rc::Rc;
use hyper::{Body, Error, Response, Server};
use hyper::service::{make_service_fn, service_fn};
use tokio::runtime::current_thread;
// Configure a runtime that runs everything on the current thread,
// which means it can spawn !Send futures...
#[tokio::main(single_thread)]
async fn main() {
fn main() {
pretty_env_logger::init();
// Configure a runtime that runs everything on the current thread
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.expect("build runtime");
// Combine it with a `LocalSet, which means it can spawn !Send futures...
let local = tokio::task::LocalSet::new();
local.block_on(&mut rt, run());
}
async fn run() {
let addr = ([127, 0, 0, 1], 3000).into();
// Using a !Send request counter is fine on 1 thread...
@@ -36,12 +48,8 @@ async fn main() {
}
});
// Since the Server needs to spawn some background tasks, we needed
// to configure an Executor that can spawn !Send futures...
let exec = current_thread::TaskExecutor::current();
let server = Server::bind(&addr)
.executor(exec)
.executor(LocalExec)
.serve(make_service);
println!("Listening on http://{}", addr);
@@ -52,3 +60,18 @@ async fn main() {
}
}
// Since the Server needs to spawn some background tasks, we needed
// to configure an Executor that can spawn !Send futures...
#[derive(Clone, Copy, Debug)]
struct LocalExec;
impl<F> hyper::rt::Executor<F> for LocalExec
where
F: std::future::Future + 'static, // not requiring `Send`
{
fn execute(&self, fut: F) {
// This will spawn into the currently running `LocalSet`.
tokio::task::spawn_local(fut);
}
}
*/

View File

@@ -21,7 +21,7 @@ impl Service<Request<Body>> for Svc {
}
fn call(&mut self, req: Request<Body>) -> Self::Future {
let mut rsp = Response::builder();
let rsp = Response::builder();
let uri = req.uri();
if uri.path() != ROOT {

View File

@@ -51,7 +51,7 @@ async fn server_upgrade(req: Request<Body>) -> Result<Response<Body>> {
// Note: This can't possibly be fulfilled until the 101 response
// is returned below, so it's better to spawn this future instead
// waiting for it to complete to then return a response.
hyper::rt::spawn(async move {
tokio::task::spawn(async move {
match req.into_body().on_upgrade().await {
Ok(upgraded) => {
if let Err(e) = server_upgraded_io(upgraded).await {
@@ -129,13 +129,13 @@ async fn main() {
// the server should be shutdown.
let (tx, rx) = oneshot::channel::<()>();
let server = server
.with_graceful_shutdown(async {
.with_graceful_shutdown(async move {
rx.await.ok();
});
// Spawn server on the default executor,
// which is usually a thread-pool from tokio default runtime.
hyper::rt::spawn(async {
tokio::task::spawn(async move {
if let Err(e) = server.await {
eprintln!("server error: {}", e);
}

View File

@@ -3,7 +3,7 @@
use hyper::{Body, Chunk, Client, Method, Request, Response, Server, StatusCode, header};
use hyper::client::HttpConnector;
use hyper::service::{make_service_fn, service_fn};
use futures_util::{TryStreamExt};
use futures_util::{StreamExt, TryStreamExt};
type GenericError = Box<dyn std::error::Error + Send + Sync>;
type Result<T> = std::result::Result<T, GenericError>;
@@ -35,13 +35,17 @@ async fn client_request_response(
Ok(Response::new(body))
}
async fn api_post_response(req: Request<Body>) -> Result<Response<Body>> {
// A web api to run against
let entire_body = req.into_body().try_concat().await?;
// TODO: Replace all unwraps with proper error handling
let str = String::from_utf8(entire_body.to_vec())?;
let mut data : serde_json::Value = serde_json::from_str(&str)?;
async fn api_post_response(mut req: Request<Body>) -> Result<Response<Body>> {
// Concatenate the body...
let mut whole_body = Vec::new();
while let Some(chunk) = req.body_mut().next().await {
whole_body.extend_from_slice(&chunk?);
}
// Decode as JSON...
let mut data: serde_json::Value = serde_json::from_slice(&whole_body)?;
// Change the JSON...
data["test"] = serde_json::Value::from("test_value");
// And respond with the new JSON.
let json = serde_json::to_string(&data)?;
let response = Response::builder()
.status(StatusCode::OK)

View File

@@ -157,11 +157,6 @@ impl Body {
Body::new(Kind::Wrapped(Box::pin(mapped)))
}
/// dox
pub async fn next(&mut self) -> Option<crate::Result<Chunk>> {
futures_util::future::poll_fn(|cx| self.poll_eof(cx)).await
}
/// Converts this `Body` into a `Future` of a pending HTTP upgrade.
///
/// See [the `upgrade` module](::upgrade) for more.
@@ -278,7 +273,7 @@ impl Body {
recv: ref mut h2, ..
} => match ready!(h2.poll_data(cx)) {
Some(Ok(bytes)) => {
let _ = h2.release_capacity().release_capacity(bytes.len());
let _ = h2.flow_control().release_capacity(bytes.len());
Poll::Ready(Some(Ok(Chunk::from(bytes))))
},
Some(Err(e)) => Poll::Ready(Some(Err(crate::Error::new_body(e)))),
@@ -528,24 +523,3 @@ impl Sender {
let _ = self.tx.try_send(Err(err));
}
}
/*
impl Sink for Sender {
type SinkItem = Chunk;
type SinkError = crate::Error;
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
Poll::Ready(Ok(()))
}
fn start_send(&mut self, msg: Chunk) -> StartSend<Self::SinkItem, Self::SinkError> {
match self.poll_ready()? {
Async::Ready(_) => {
self.send_data(msg).map_err(|_| crate::Error::new_closed())?;
Ok(AsyncSink::Ready)
}
Async::NotReady => Ok(AsyncSink::NotReady(msg)),
}
}
}
*/

View File

@@ -137,13 +137,6 @@ impl IntoIterator for Chunk {
}
}
impl Extend<u8> for Chunk {
#[inline]
fn extend<T>(&mut self, iter: T) where T: IntoIterator<Item=u8> {
self.bytes.extend(iter)
}
}
impl Iterator for IntoIter {
type Item = u8;

View File

@@ -15,6 +15,9 @@
//! client responses). It is also a decent default implementation if you don't
//! have very custom needs of your send streams.
#[doc(hidden)]
pub use http_body::Body as HttpBody;
pub use self::body::{Body, Sender};
pub use self::chunk::Chunk;
pub use self::payload::Payload;

View File

@@ -14,11 +14,11 @@ use std::sync::Arc;
use bytes::Bytes;
use futures_util::future::{self, Either, FutureExt as _};
use pin_project::{pin_project, project};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use tower_service::Service;
use crate::body::Payload;
use crate::common::{Exec, Future, Pin, Poll, task};
use crate::common::{BoxSendFuture, Exec, Executor, Future, Pin, Poll, task};
use crate::upgrade::Upgraded;
use crate::proto;
use super::dispatch;
@@ -458,8 +458,7 @@ impl Builder {
/// Provide an executor to execute background HTTP2 tasks.
pub fn executor<E>(&mut self, exec: E) -> &mut Builder
where
for<'a> &'a E: tokio_executor::Executor,
E: Send + Sync + 'static,
E: Executor<BoxSendFuture> + Send + Sync + 'static,
{
self.exec = Exec::Executor(Arc::new(exec));
self

View File

@@ -30,6 +30,7 @@ use std::net::{
};
use std::str::FromStr;
use tokio::task::JoinHandle;
use tower_service::Service;
use crate::common::{Future, Pin, Poll, task};
@@ -54,7 +55,7 @@ pub struct GaiAddrs {
/// A future to resolve a name returned by `GaiResolver`.
pub struct GaiFuture {
inner: tokio_executor::blocking::Blocking<Result<IpAddrs, io::Error>>,
inner: JoinHandle<Result<IpAddrs, io::Error>>,
}
impl Name {
@@ -123,7 +124,7 @@ impl Service<Name> for GaiResolver {
}
fn call(&mut self, name: Name) -> Self::Future {
let blocking = tokio_executor::blocking::run(move || {
let blocking = tokio::task::spawn_blocking(move || {
debug!("resolving host={:?}", name.host);
(&*name.host, 0).to_socket_addrs()
.map(|i| IpAddrs { iter: i })
@@ -146,8 +147,9 @@ impl Future for GaiFuture {
fn poll(mut self: Pin<&mut Self>, cx: &mut task::Context<'_>) -> Poll<Self::Output> {
Pin::new(&mut self.inner).poll(cx).map(|res| match res {
Ok(addrs) => Ok(GaiAddrs { inner: addrs }),
Err(err) => Err(err),
Ok(Ok(addrs)) => Ok(GaiAddrs { inner: addrs }),
Ok(Err(err)) => Err(err),
Err(join_err) => panic!("gai background task failed: {:?}", join_err),
})
}
}
@@ -232,6 +234,7 @@ impl Iterator for IpAddrs {
}
}
/*
/// A resolver using `getaddrinfo` calls via the `tokio_executor::threadpool::blocking` API.
///
/// Unlike the `GaiResolver` this will not spawn dedicated threads, but only works when running on the
@@ -286,6 +289,7 @@ impl Future for TokioThreadpoolGaiFuture {
}
}
}
*/
mod sealed {
use tower_service::Service;

View File

@@ -7,17 +7,16 @@ use std::sync::Arc;
use std::time::Duration;
use http::uri::{Scheme, Uri};
use futures_util::{TryFutureExt, FutureExt};
use futures_util::{TryFutureExt};
use net2::TcpBuilder;
use pin_project::{pin_project, project};
use tokio_net::driver::Handle;
use tokio_net::tcp::TcpStream;
use tokio_timer::{Delay, Timeout};
use tokio::net::TcpStream;
use tokio::time::Delay;
use crate::common::{Future, Pin, Poll, task};
use super::{Connected, Destination};
use super::dns::{self, GaiResolver, Resolve};
#[cfg(feature = "runtime")] use super::dns::TokioThreadpoolGaiResolver;
//#[cfg(feature = "runtime")] use super::dns::TokioThreadpoolGaiResolver;
// TODO: unbox me?
type ConnectFuture = Pin<Box<dyn Future<Output = io::Result<TcpStream>> + Send>>;
@@ -73,7 +72,6 @@ pub struct HttpInfo {
struct Config {
connect_timeout: Option<Duration>,
enforce_http: bool,
handle: Option<Handle>,
happy_eyeballs_timeout: Option<Duration>,
keep_alive_timeout: Option<Duration>,
local_address: Option<IpAddr>,
@@ -92,6 +90,7 @@ impl HttpConnector {
}
}
/*
#[cfg(feature = "runtime")]
impl HttpConnector<TokioThreadpoolGaiResolver> {
/// Construct a new HttpConnector using the `TokioThreadpoolGaiResolver`.
@@ -101,6 +100,7 @@ impl HttpConnector<TokioThreadpoolGaiResolver> {
HttpConnector::new_with_resolver(TokioThreadpoolGaiResolver::new())
}
}
*/
impl<R> HttpConnector<R> {
@@ -112,7 +112,6 @@ impl<R> HttpConnector<R> {
config: Arc::new(Config {
connect_timeout: None,
enforce_http: true,
handle: None,
happy_eyeballs_timeout: Some(Duration::from_millis(300)),
keep_alive_timeout: None,
local_address: None,
@@ -133,14 +132,6 @@ impl<R> HttpConnector<R> {
self.config_mut().enforce_http = is_enforced;
}
/// Set a handle to a `Reactor` to register connections to.
///
/// If `None`, the implicit default reactor will be used.
#[inline]
pub fn set_reactor(&mut self, handle: Option<Handle>) {
self.config_mut().handle = handle;
}
/// Set that all sockets have `SO_KEEPALIVE` set with the supplied duration.
///
/// If `None`, the option will not be set.
@@ -276,10 +267,10 @@ where
);
if self.config.enforce_http {
if dst.uri.scheme_part() != Some(&Scheme::HTTP) {
if dst.uri.scheme() != Some(&Scheme::HTTP) {
return self.invalid_url(INVALID_NOT_HTTP);
}
} else if dst.uri.scheme_part().is_none() {
} else if dst.uri.scheme().is_none() {
return self.invalid_url(INVALID_MISSING_SCHEME);
}
@@ -287,9 +278,9 @@ where
Some(s) => s,
None => return self.invalid_url(INVALID_MISSING_HOST),
};
let port = match dst.uri.port_part() {
let port = match dst.uri.port() {
Some(port) => port.as_u16(),
None => if dst.uri.scheme_part() == Some(&Scheme::HTTPS) { 443 } else { 80 },
None => if dst.uri.scheme() == Some(&Scheme::HTTPS) { 443 } else { 80 },
};
HttpConnecting {
@@ -314,10 +305,7 @@ where
}
fn call(&mut self, uri: Uri) -> Self::Future {
self
.call(Destination { uri })
.map_ok(|(s, _)| s)
.boxed()
Box::pin(self.call(Destination { uri }).map_ok(|(s, _)| s))
}
}
@@ -447,7 +435,7 @@ impl<R: Resolve> Future for HttpConnecting<R> {
config.local_address, addrs, config.connect_timeout, config.happy_eyeballs_timeout, config.reuse_address));
},
State::Connecting(ref mut c) => {
let sock = ready!(c.poll(cx, &config.handle))
let sock = ready!(c.poll(cx))
.map_err(ConnectError::m("tcp connect error"))?;
if let Some(dur) = config.keep_alive_timeout {
@@ -515,7 +503,7 @@ impl ConnectingTcp {
local_addr,
preferred: ConnectingTcpRemote::new(preferred_addrs, connect_timeout),
fallback: Some(ConnectingTcpFallback {
delay: tokio_timer::delay_for(fallback_timeout),
delay: tokio::time::delay_for(fallback_timeout),
remote: ConnectingTcpRemote::new(fallback_addrs, connect_timeout),
}),
reuse_address,
@@ -555,12 +543,10 @@ impl ConnectingTcpRemote {
}
impl ConnectingTcpRemote {
// not a Future, since passing a &Handle to poll
fn poll(
&mut self,
cx: &mut task::Context<'_>,
local_addr: &Option<IpAddr>,
handle: &Option<Handle>,
reuse_address: bool,
) -> Poll<io::Result<TcpStream>> {
let mut err = None;
@@ -577,14 +563,14 @@ impl ConnectingTcpRemote {
err = Some(e);
if let Some(addr) = self.addrs.next() {
debug!("connecting to {}", addr);
*current = connect(&addr, local_addr, handle, reuse_address, self.connect_timeout)?;
*current = connect(&addr, local_addr, reuse_address, self.connect_timeout)?;
continue;
}
}
}
} else if let Some(addr) = self.addrs.next() {
debug!("connecting to {}", addr);
self.current = Some(connect(&addr, local_addr, handle, reuse_address, self.connect_timeout)?);
self.current = Some(connect(&addr, local_addr, reuse_address, self.connect_timeout)?);
continue;
}
@@ -596,7 +582,6 @@ impl ConnectingTcpRemote {
fn connect(
addr: &SocketAddr,
local_addr: &Option<IpAddr>,
handle: &Option<Handle>,
reuse_address: bool,
connect_timeout: Option<Duration>,
) -> io::Result<ConnectFuture> {
@@ -625,18 +610,14 @@ fn connect(
builder.bind(any)?;
}
let handle = match *handle {
Some(ref handle) => handle.clone(),
None => Handle::default(),
};
let addr = *addr;
let std_tcp = builder.to_tcp_stream()?;
Ok(Box::pin(async move {
let connect = TcpStream::connect_std(std_tcp, &addr, &handle);
let connect = TcpStream::connect_std(std_tcp, &addr);
match connect_timeout {
Some(timeout) => match Timeout::new(connect, timeout).await {
Some(dur) => match tokio::time::timeout(dur, connect).await {
Ok(Ok(s)) => Ok(s),
Ok(Err(e)) => Err(e),
Err(e) => Err(io::Error::new(io::ErrorKind::TimedOut, e)),
@@ -647,16 +628,16 @@ fn connect(
}
impl ConnectingTcp {
fn poll(&mut self, cx: &mut task::Context<'_>, handle: &Option<Handle>) -> Poll<io::Result<TcpStream>> {
fn poll(&mut self, cx: &mut task::Context<'_>) -> Poll<io::Result<TcpStream>> {
match self.fallback.take() {
None => self.preferred.poll(cx, &self.local_addr, handle, self.reuse_address),
Some(mut fallback) => match self.preferred.poll(cx, &self.local_addr, handle, self.reuse_address) {
None => self.preferred.poll(cx, &self.local_addr, self.reuse_address),
Some(mut fallback) => match self.preferred.poll(cx, &self.local_addr, self.reuse_address) {
Poll::Ready(Ok(stream)) => {
// Preferred successful - drop fallback.
Poll::Ready(Ok(stream))
}
Poll::Pending => match Pin::new(&mut fallback.delay).poll(cx) {
Poll::Ready(()) => match fallback.remote.poll(cx, &self.local_addr, handle, self.reuse_address) {
Poll::Ready(()) => match fallback.remote.poll(cx, &self.local_addr, self.reuse_address) {
Poll::Ready(Ok(stream)) => {
// Fallback successful - drop current preferred,
// but keep fallback as new preferred.
@@ -682,7 +663,7 @@ impl ConnectingTcp {
Poll::Ready(Err(_)) => {
// Preferred failed - use fallback as new preferred.
self.preferred = fallback.remote;
self.preferred.poll(cx, &self.local_addr, handle, self.reuse_address)
self.preferred.poll(cx, &self.local_addr, self.reuse_address)
}
}
}
@@ -693,8 +674,6 @@ impl ConnectingTcp {
mod tests {
use std::io;
use tokio_net::driver::Handle;
use super::{Connected, Destination, HttpConnector};
use super::super::sealed::Connect;
@@ -738,8 +717,6 @@ mod tests {
use std::task::Poll;
use std::time::{Duration, Instant};
use tokio::runtime::current_thread::Runtime;
use crate::common::{Pin, task};
use super::dns;
use super::ConnectingTcp;
@@ -748,7 +725,12 @@ mod tests {
let server4 = TcpListener::bind("127.0.0.1:0").unwrap();
let addr = server4.local_addr().unwrap();
let _server6 = TcpListener::bind(&format!("[::1]:{}", addr.port())).unwrap();
let mut rt = Runtime::new().unwrap();
let mut rt = tokio::runtime::Builder::new()
.enable_io()
.enable_time()
.basic_scheduler()
.build()
.unwrap();
let local_timeout = Duration::default();
let unreachable_v4_timeout = measure_connect(unreachable_ipv4_addr()).1;
@@ -804,12 +786,13 @@ mod tests {
}
let addrs = hosts.iter().map(|host| (host.clone(), addr.port()).into()).collect();
let (res, duration) = rt.block_on(async move {
let connecting_tcp = ConnectingTcp::new(None, dns::IpAddrs::new(addrs), None, Some(fallback_timeout), false);
let fut = ConnectingTcpFuture(connecting_tcp);
let start = Instant::now();
let res = rt.block_on(fut).unwrap();
let duration = start.elapsed();
let res = fut.await.unwrap();
(res, start.elapsed())
});
// Allow actual duration to be +/- 150ms off.
let min_duration = if timeout >= Duration::from_millis(150) {
@@ -830,7 +813,7 @@ mod tests {
type Output = Result<u8, std::io::Error>;
fn poll(mut self: Pin<&mut Self>, cx: &mut task::Context<'_>) -> Poll<Self::Output> {
match self.0.poll(cx,&Some(Handle::default())) {
match self.0.poll(cx) {
Poll::Ready(Ok(stream)) => Poll::Ready(Ok(
if stream.peer_addr().unwrap().is_ipv4() { 4 } else { 6 }
)),

View File

@@ -50,8 +50,8 @@ impl Destination {
/// Returns an error if the uri contains no authority or
/// no scheme.
pub fn try_from_uri(uri: Uri) -> crate::Result<Self> {
uri.authority_part().ok_or(crate::error::Parse::Uri)?;
uri.scheme_part().ok_or(crate::error::Parse::Uri)?;
uri.authority().ok_or(crate::error::Parse::Uri)?;
uri.scheme().ok_or(crate::error::Parse::Uri)?;
Ok(Destination { uri })
}
@@ -131,11 +131,11 @@ impl Destination {
}
let auth = if let Some(port) = self.port() {
let bytes = Bytes::from(format!("{}:{}", host, port));
uri::Authority::from_shared(bytes)
uri::Authority::from_maybe_shared(bytes)
.map_err(crate::error::Parse::from)?
} else {
let auth = host.parse::<uri::Authority>().map_err(crate::error::Parse::from)?;
if auth.port_part().is_some() { // std::uri::Authority::Uri
if auth.port().is_some() { // std::uri::Authority::Uri
return Err(crate::error::Parse::Uri.into());
}
auth
@@ -186,7 +186,7 @@ impl Destination {
write!(buf, "{}", port)
.expect("should have space for 5 digits");
uri::Authority::from_shared(buf.freeze())
uri::Authority::from_maybe_shared(buf.freeze())
.expect("valid host + :port should be valid authority")
} else {
self.host().parse()
@@ -372,7 +372,7 @@ where
pub(super) mod sealed {
use std::error::Error as StdError;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::common::{Future, Unpin};
use super::{Connected, Destination};

View File

@@ -253,8 +253,6 @@ mod tests {
use std::pin::Pin;
use std::task::{Context, Poll};
use tokio::runtime::current_thread::Runtime;
use super::{Callback, channel, Receiver};
#[derive(Debug)]
@@ -285,23 +283,18 @@ mod tests {
}
}
#[test]
fn drop_receiver_sends_cancel_errors() {
#[tokio::test]
async fn drop_receiver_sends_cancel_errors() {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let (mut tx, mut rx) = channel::<Custom, ()>();
// must poll once for try_send to succeed
rt.block_on(async {
let poll_once = PollOnce(&mut rx);
assert!(poll_once.await.is_none(), "rx empty");
});
assert!(PollOnce(&mut rx).await.is_none(), "rx empty");
let promise = tx.try_send(Custom(43)).unwrap();
drop(rx);
rt.block_on(async {
let fulfilled = promise.await;
let err = fulfilled
.expect("fulfilled")
@@ -310,31 +303,23 @@ mod tests {
(&crate::error::Kind::Canceled, Some(_)) => (),
e => panic!("expected Error::Cancel(_), found {:?}", e),
}
});
}
#[test]
fn sender_checks_for_want_on_send() {
let mut rt = Runtime::new().unwrap();
#[tokio::test]
async fn sender_checks_for_want_on_send() {
let (mut tx, mut rx) = channel::<Custom, ()>();
// one is allowed to buffer, second is rejected
let _ = tx.try_send(Custom(1)).expect("1 buffered");
tx.try_send(Custom(2)).expect_err("2 not ready");
rt.block_on(async {
let poll_once = PollOnce(&mut rx);
assert!(poll_once.await.is_some(), "rx empty");
});
assert!(PollOnce(&mut rx).await.is_some(), "rx once");
// Even though 1 has been popped, only 1 could be buffered for the
// lifetime of the channel.
tx.try_send(Custom(2)).expect_err("2 still not ready");
rt.block_on(async {
let poll_once = PollOnce(&mut rx);
assert!(poll_once.await.is_none(), "rx empty");
});
assert!(PollOnce(&mut rx).await.is_none(), "rx empty");
let _ = tx.try_send(Custom(2)).expect("2 ready");
}
@@ -358,7 +343,11 @@ mod tests {
fn giver_queue_throughput(b: &mut test::Bencher) {
use crate::{Body, Request, Response};
let mut rt = Runtime::new().unwrap();
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.unwrap();
let (mut tx, mut rx) = channel::<Request<Body>, Response<Body>>();
b.iter(move || {
@@ -378,7 +367,11 @@ mod tests {
#[cfg(feature = "nightly")]
#[bench]
fn giver_queue_not_ready(b: &mut test::Bencher) {
let mut rt = Runtime::new().unwrap();
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.unwrap();
let (_tx, mut rx) = channel::<i32, ()>();
b.iter(move || {
rt.block_on(async {

View File

@@ -27,7 +27,7 @@
//! [full client example](https://github.com/hyperium/hyper/blob/master/examples/client.rs).
//!
//! ```
//! use hyper::{Client, Uri};
//! use hyper::{body::HttpBody as _, Client, Uri};
//!
//! # #[cfg(feature = "tcp")]
//! # async fn fetch_httpbin() -> hyper::Result<()> {
@@ -70,7 +70,7 @@ use http::header::{HeaderValue, HOST};
use http::uri::Scheme;
use crate::body::{Body, Payload};
use crate::common::{lazy as hyper_lazy, Lazy, Future, Pin, Poll, task};
use crate::common::{lazy as hyper_lazy, BoxSendFuture, Executor, Lazy, Future, Pin, Poll, task};
use self::connect::{Alpn, sealed::Connect, Connected, Destination};
use self::pool::{Key as PoolKey, Pool, Poolable, Pooled, Reservation};
@@ -285,10 +285,9 @@ where C: Connect + Clone + Send + Sync + 'static,
req
.headers_mut()
.entry(HOST)
.expect("HOST is always valid header name")
.or_insert_with(|| {
let hostname = uri.host().expect("authority implies host");
if let Some(port) = uri.port_part() {
if let Some(port) = uri.port() {
let s = format!("{}:{}", hostname, port);
HeaderValue::from_str(&s)
} else {
@@ -359,10 +358,7 @@ where C: Connect + Clone + Send + Sync + 'static,
drop(delayed_tx);
});
if let Err(err) = executor.execute(on_idle) {
// This task isn't critical, so just log and ignore.
warn!("error spawning task to insert idle connection: {}", err);
}
executor.execute(on_idle);
} else {
// There's no body to delay, but the connection isn't
// ready yet. Only re-insert when it's ready
@@ -371,10 +367,7 @@ where C: Connect + Clone + Send + Sync + 'static,
})
.map(|_| ());
if let Err(err) = executor.execute(on_idle) {
// This task isn't critical, so just log and ignore.
warn!("error spawning task to insert idle connection: {}", err);
}
executor.execute(on_idle);
}
res
})))
@@ -513,20 +506,13 @@ where C: Connect + Clone + Send + Sync + 'static,
.handshake(io)
.and_then(move |(tx, conn)| {
trace!("handshake complete, spawning background dispatcher task");
let bg = executor.execute(conn.map_err(|e| {
executor.execute(conn.map_err(|e| {
debug!("client connection error: {}", e)
}).map(|_| ()));
// This task is critical, so an execute error
// should be returned.
if let Err(err) = bg {
warn!("error spawning critical client task: {}", err);
return Either::Left(future::err(err));
}
// Wait for 'conn' to ready up before we
// declare this tx as usable
Either::Right(tx.when_ready())
tx.when_ready()
})
.map_ok(move |tx| {
pool.pooled(connecting, PoolClient {
@@ -742,12 +728,12 @@ fn origin_form(uri: &mut Uri) {
}
fn absolute_form(uri: &mut Uri) {
debug_assert!(uri.scheme_part().is_some(), "absolute_form needs a scheme");
debug_assert!(uri.authority_part().is_some(), "absolute_form needs an authority");
debug_assert!(uri.scheme().is_some(), "absolute_form needs a scheme");
debug_assert!(uri.authority().is_some(), "absolute_form needs an authority");
// If the URI is to HTTPS, and the connector claimed to be a proxy,
// then it *should* have tunneled, and so we don't want to send
// absolute-form in that case.
if uri.scheme_part() == Some(&Scheme::HTTPS) {
if uri.scheme() == Some(&Scheme::HTTPS) {
origin_form(uri);
}
}
@@ -765,7 +751,7 @@ fn authority_form(uri: &mut Uri) {
}
}
}
*uri = match uri.authority_part() {
*uri = match uri.authority() {
Some(auth) => {
let mut parts = ::http::uri::Parts::default();
parts.authority = Some(auth.clone());
@@ -779,14 +765,13 @@ fn authority_form(uri: &mut Uri) {
fn extract_domain(uri: &mut Uri, is_http_connect: bool) -> crate::Result<String> {
let uri_clone = uri.clone();
match (uri_clone.scheme_part(), uri_clone.authority_part()) {
match (uri_clone.scheme(), uri_clone.authority()) {
(Some(scheme), Some(auth)) => {
Ok(format!("{}://{}", scheme, auth))
}
(None, Some(auth)) if is_http_connect => {
let port = auth.port_part();
let scheme = match port.as_ref().map(|p| p.as_str()) {
Some("443") => {
let scheme = match auth.port_u16() {
Some(443) => {
set_scheme(uri, Scheme::HTTPS);
"https"
}
@@ -805,7 +790,7 @@ fn extract_domain(uri: &mut Uri, is_http_connect: bool) -> crate::Result<String>
}
fn set_scheme(uri: &mut Uri, scheme: Scheme) {
debug_assert!(uri.scheme_part().is_none(), "set_scheme expects no existing scheme");
debug_assert!(uri.scheme().is_none(), "set_scheme expects no existing scheme");
let old = mem::replace(uri, Uri::default());
let mut parts: ::http::uri::Parts = old.into();
parts.scheme = Some(scheme);
@@ -1013,8 +998,7 @@ impl Builder {
/// Provide an executor to execute background `Connection` tasks.
pub fn executor<E>(&mut self, exec: E) -> &mut Self
where
for<'a> &'a E: tokio_executor::Executor,
E: Send + Sync + 'static,
E: Executor<BoxSendFuture> + Send + Sync + 'static,
{
self.conn_builder.executor(exec);
self

View File

@@ -6,7 +6,7 @@ use std::time::{Duration, Instant};
use futures_channel::oneshot;
#[cfg(feature = "runtime")]
use tokio_timer::Interval;
use tokio::time::Interval;
use crate::common::{Exec, Future, Pin, Poll, Unpin, task};
use super::Ver;
@@ -414,18 +414,13 @@ impl<T: Poolable> PoolInner<T> {
}
};
let start = Instant::now() + dur;
let interval = IdleTask {
interval: Interval::new(start, dur),
interval: tokio::time::interval(dur),
pool: WeakOpt::downgrade(pool_ref),
pool_drop_notifier: rx,
};
if let Err(err) = self.exec.execute(interval) {
// This task isn't critical, so simply log and ignore.
warn!("error spawning connection pool idle interval: {}", err);
}
self.exec.execute(interval);
}
}
@@ -743,7 +738,7 @@ impl<T: Poolable + 'static> Future for IdleTask<T> {
}
}
ready!(Pin::new(&mut self.interval).poll_next(cx));
ready!(self.interval.poll_tick(cx));
if let Some(inner) = self.pool.upgrade() {
if let Ok(mut inner) = inner.lock() {
@@ -779,8 +774,6 @@ mod tests {
use std::task::Poll;
use std::time::Duration;
use tokio::runtime::current_thread::Runtime;
use crate::common::{Exec, Future, Pin, task};
use super::{Connecting, Key, Poolable, Pool, Reservation, WeakOpt};
@@ -825,21 +818,18 @@ mod tests {
pool
}
#[test]
fn test_pool_checkout_smoke() {
let mut rt = Runtime::new().unwrap();
#[tokio::test]
async fn test_pool_checkout_smoke() {
let pool = pool_no_timer();
let key = Arc::new("foo".to_string());
let pooled = pool.pooled(c(key.clone()), Uniq(41));
drop(pooled);
rt.block_on(async {
match pool.checkout(key).await {
Ok(pooled) => assert_eq!(*pooled, Uniq(41)),
Err(_) => panic!("not ready"),
};
})
}
/// Helper to check if the future is ready after polling once.
@@ -859,27 +849,23 @@ mod tests {
}
}
#[test]
fn test_pool_checkout_returns_none_if_expired() {
let mut rt = Runtime::new().unwrap();
#[tokio::test]
async fn test_pool_checkout_returns_none_if_expired() {
let pool = pool_no_timer();
let key = Arc::new("foo".to_string());
let pooled = pool.pooled(c(key.clone()), Uniq(41));
drop(pooled);
std::thread::sleep(pool.locked().timeout.unwrap());
rt.block_on(async {
tokio::time::delay_for(pool.locked().timeout.unwrap()).await;
let mut checkout = pool.checkout(key);
let poll_once = PollOnce(&mut checkout);
let is_not_ready = poll_once.await.is_none();
assert!(is_not_ready);
});
}
#[cfg(feature = "runtime")]
#[test]
fn test_pool_checkout_removes_expired() {
let mut rt = Runtime::new().unwrap();
#[tokio::test]
async fn test_pool_checkout_removes_expired() {
let pool = pool_no_timer();
let key = Arc::new("foo".to_string());
@@ -888,15 +874,13 @@ mod tests {
pool.pooled(c(key.clone()), Uniq(99));
assert_eq!(pool.locked().idle.get(&key).map(|entries| entries.len()), Some(3));
std::thread::sleep(pool.locked().timeout.unwrap());
tokio::time::delay_for(pool.locked().timeout.unwrap()).await;
rt.block_on(async {
let mut checkout = pool.checkout(key.clone());
let poll_once = PollOnce(&mut checkout);
// checkout.await should clean out the expired
poll_once.await;
assert!(pool.locked().idle.get(&key).is_none());
});
}
#[test]
@@ -913,14 +897,11 @@ mod tests {
}
#[cfg(feature = "runtime")]
#[test]
fn test_pool_timer_removes_expired() {
use std::time::Instant;
use tokio_timer::delay;
let mut rt = Runtime::new().unwrap();
#[tokio::test]
async fn test_pool_timer_removes_expired() {
let pool = Pool::new(super::Config {
enabled: true,
keep_alive_timeout: Some(Duration::from_millis(100)),
keep_alive_timeout: Some(Duration::from_millis(10)),
max_idle_per_host: ::std::usize::MAX,
},
&Exec::Default,
@@ -928,32 +909,23 @@ mod tests {
let key = Arc::new("foo".to_string());
// Since pool.pooled() will be calling spawn on drop, need to be sure
// those drops are called while `rt` is the current executor. To do so,
// call those inside a future.
rt.block_on(async {
pool.pooled(c(key.clone()), Uniq(41));
pool.pooled(c(key.clone()), Uniq(5));
pool.pooled(c(key.clone()), Uniq(99));
});
assert_eq!(pool.locked().idle.get(&key).map(|entries| entries.len()), Some(3));
// Let the timer tick passed the expiration...
rt.block_on(async {
let deadline = Instant::now() + Duration::from_millis(200);
delay(deadline).await;
});
tokio::time::delay_for(Duration::from_millis(50)).await;
assert!(pool.locked().idle.get(&key).is_none());
}
#[test]
fn test_pool_checkout_task_unparked() {
#[tokio::test]
async fn test_pool_checkout_task_unparked() {
use futures_util::future::join;
use futures_util::FutureExt;
let mut rt = Runtime::new().unwrap();
let pool = pool_no_timer();
let key = Arc::new("foo".to_string());
let pooled = pool.pooled(c(key.clone()), Uniq(41));
@@ -970,14 +942,11 @@ mod tests {
},
).map(|(entry, _)| entry);
rt.block_on(async {
assert_eq!(*checkout.await.unwrap(), Uniq(41));
});
}
#[test]
fn test_pool_checkout_drop_cleans_up_waiters() {
let mut rt = Runtime::new().unwrap();
#[tokio::test]
async fn test_pool_checkout_drop_cleans_up_waiters() {
let pool = pool_no_timer::<Uniq<i32>>();
let key = Arc::new("localhost:12345".to_string());
@@ -988,12 +957,10 @@ mod tests {
let poll_once2 = PollOnce(&mut checkout2);
// first poll needed to get into Pool's parked
rt.block_on(async {
poll_once1.await;
assert_eq!(pool.locked().waiters.get(&key).unwrap().len(), 1);
poll_once2.await;
assert_eq!(pool.locked().waiters.get(&key).unwrap().len(), 2);
});
// on drop, clean up Pool
drop(checkout1);

View File

@@ -63,7 +63,7 @@ where
if let Err(e) = conn.await {
debug!("connection error: {:?}", e);
}
})?;
});
Ok(sr)
},
Err(e) => Err(e)

View File

@@ -1,34 +0,0 @@
use bytes::Buf;
use iovec::IoVec;
/// A `Buf` wrapping a static byte slice.
#[derive(Debug)]
pub(crate) struct StaticBuf(pub(crate) &'static [u8]);
impl Buf for StaticBuf {
#[inline]
fn remaining(&self) -> usize {
self.0.len()
}
#[inline]
fn bytes(&self) -> &[u8] {
self.0
}
#[inline]
fn advance(&mut self, cnt: usize) {
self.0 = &self.0[cnt..];
}
#[inline]
fn bytes_vec<'t>(&'t self, dst: &mut [&'t IoVec]) -> usize {
if dst.is_empty() || self.0.is_empty() {
0
} else {
dst[0] = self.0.into();
1
}
}
}

View File

@@ -1,12 +1,12 @@
use std::mem;
use futures_util::FutureExt as _;
use tokio_sync::{mpsc, watch};
use tokio::sync::{mpsc, watch};
use pin_project::pin_project;
use super::{Future, Never, Poll, Pin, task};
// Sentinel value signaling that the watch is still open
#[derive(Clone, Copy)]
enum Action {
Open,
// Closed isn't sent via the `Action` type, but rather once
@@ -103,10 +103,7 @@ where
loop {
match mem::replace(me.state, State::Draining) {
State::Watch(on_drain) => {
let recv = me.watch.rx.recv_ref();
futures_util::pin_mut!(recv);
match recv.poll_unpin(cx) {
match me.watch.rx.poll_recv_ref(cx) {
Poll::Ready(None) => {
// Drain has been triggered!
on_drain(me.future.as_mut());
@@ -151,7 +148,8 @@ mod tests {
#[test]
fn watch() {
tokio_test::task::mock(|cx| {
let mut mock = tokio_test::task::spawn(());
mock.enter(|cx, _| {
let (tx, rx) = channel();
let fut = TestMe {
draining: false,
@@ -198,7 +196,8 @@ mod tests {
#[test]
fn watch_clones() {
tokio_test::task::mock(|cx| {
let mut mock = tokio_test::task::spawn(());
mock.enter(|cx, _| {
let (tx, rx) = channel();
let fut1 = TestMe {

View File

@@ -3,48 +3,39 @@ use std::future::Future;
use std::pin::Pin;
use std::sync::Arc;
use tokio_executor::{SpawnError, TypedExecutor};
use crate::body::{Payload, Body};
use crate::proto::h2::server::H2Stream;
use crate::server::conn::spawn_all::{NewSvcTask, Watcher};
use crate::service::HttpService;
/// An executor of futures.
pub trait Executor<Fut> {
/// Place the future into the executor to be run.
fn execute(&self, fut: Fut);
}
pub trait H2Exec<F, B: Payload>: Clone {
fn execute_h2stream(&mut self, fut: H2Stream<F, B>) -> crate::Result<()>;
fn execute_h2stream(&mut self, fut: H2Stream<F, B>);
}
pub trait NewSvcExec<I, N, S: HttpService<Body>, E, W: Watcher<I, S, E>>: Clone {
fn execute_new_svc(&mut self, fut: NewSvcTask<I, N, S, E, W>) -> crate::Result<()>;
fn execute_new_svc(&mut self, fut: NewSvcTask<I, N, S, E, W>);
}
type BoxFuture = Pin<Box<dyn Future<Output=()> + Send>>;
pub trait SharedExecutor {
fn shared_spawn(&self, future: BoxFuture) -> Result<(), SpawnError>;
}
impl<E> SharedExecutor for E
where
for<'a> &'a E: tokio_executor::Executor,
{
fn shared_spawn(mut self: &Self, future: BoxFuture) -> Result<(), SpawnError> {
tokio_executor::Executor::spawn(&mut self, future)
}
}
pub type BoxSendFuture = Pin<Box<dyn Future<Output=()> + Send>>;
// Either the user provides an executor for background tasks, or we use
// `tokio::spawn`.
#[derive(Clone)]
pub enum Exec {
Default,
Executor(Arc<dyn SharedExecutor + Send + Sync>),
Executor(Arc<dyn Executor<BoxSendFuture> + Send + Sync>),
}
// ===== impl Exec =====
impl Exec {
pub(crate) fn execute<F>(&self, fut: F) -> crate::Result<()>
pub(crate) fn execute<F>(&self, fut: F)
where
F: Future<Output=()> + Send + 'static,
{
@@ -52,34 +43,7 @@ impl Exec {
Exec::Default => {
#[cfg(feature = "tcp")]
{
use std::error::Error as StdError;
struct TokioSpawnError;
impl fmt::Debug for TokioSpawnError {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
fmt::Debug::fmt("tokio::spawn failed (is a tokio runtime running this future?)", f)
}
}
impl fmt::Display for TokioSpawnError {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
fmt::Display::fmt("tokio::spawn failed (is a tokio runtime running this future?)", f)
}
}
impl StdError for TokioSpawnError {
fn description(&self) -> &str {
"tokio::spawn failed"
}
}
::tokio_executor::DefaultExecutor::current()
.spawn(Box::pin(fut))
.map_err(|err| {
warn!("executor error: {:?}", err);
crate::Error::new_execute(TokioSpawnError)
})
tokio::task::spawn(fut);
}
#[cfg(not(feature = "tcp"))]
{
@@ -88,11 +52,7 @@ impl Exec {
}
},
Exec::Executor(ref e) => {
e.shared_spawn(Box::pin(fut))
.map_err(|err| {
warn!("executor error: {:?}", err);
crate::Error::new_execute("custom executor failed")
})
e.execute(Box::pin(fut));
},
}
}
@@ -111,7 +71,7 @@ where
H2Stream<F, B>: Future<Output = ()> + Send + 'static,
B: Payload,
{
fn execute_h2stream(&mut self, fut: H2Stream<F, B>) -> crate::Result<()> {
fn execute_h2stream(&mut self, fut: H2Stream<F, B>) {
self.execute(fut)
}
}
@@ -122,7 +82,7 @@ where
S: HttpService<Body>,
W: Watcher<I, S, E>,
{
fn execute_new_svc(&mut self, fut: NewSvcTask<I, N, S, E, W>) -> crate::Result<()> {
fn execute_new_svc(&mut self, fut: NewSvcTask<I, N, S, E, W>) {
self.execute(fut)
}
}
@@ -131,34 +91,24 @@ where
impl<E, F, B> H2Exec<F, B> for E
where
E: TypedExecutor<H2Stream<F, B>> + Clone,
E: Executor<H2Stream<F, B>> + Clone,
H2Stream<F, B>: Future<Output=()>,
B: Payload,
{
fn execute_h2stream(&mut self, fut: H2Stream<F, B>) -> crate::Result<()> {
self.spawn(fut)
.map_err(|err| {
warn!("executor error: {:?}", err);
crate::Error::new_execute("custom executor failed")
})
fn execute_h2stream(&mut self, fut: H2Stream<F, B>) {
self.execute(fut)
}
}
impl<I, N, S, E, W> NewSvcExec<I, N, S, E, W> for E
where
E: TypedExecutor<NewSvcTask<I, N, S, E, W>> + Clone,
E: Executor<NewSvcTask<I, N, S, E, W>> + Clone,
NewSvcTask<I, N, S, E, W>: Future<Output=()>,
S: HttpService<Body>,
W: Watcher<I, S, E>,
{
fn execute_new_svc(&mut self, fut: NewSvcTask<I, N, S, E, W>) -> crate::Result<()> {
self.spawn(fut)
.map_err(|err| {
warn!("executor error: {:?}", err);
crate::Error::new_execute("custom executor failed")
})
fn execute_new_svc(&mut self, fut: NewSvcTask<I, N, S, E, W>) {
self.execute(fut)
}
}
// ===== StdError impls =====

View File

@@ -1,8 +1,8 @@
use std::io::{self, Read};
use std::{cmp, io};
use std::marker::Unpin;
use bytes::{Buf, Bytes, IntoBuf};
use tokio_io::{AsyncRead, AsyncWrite};
use bytes::{Buf, Bytes};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::common::{Pin, Poll, task};
@@ -43,26 +43,22 @@ where
T: AsyncRead + Unpin,
{
#[inline]
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool {
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [std::mem::MaybeUninit<u8>]) -> bool {
self.inner.prepare_uninitialized_buffer(buf)
}
fn poll_read(mut self: Pin<&mut Self>, cx: &mut task::Context<'_>, buf: &mut [u8]) -> Poll<io::Result<usize>> {
if let Some(pre_bs) = self.pre.take() {
if let Some(mut prefix) = self.pre.take() {
// If there are no remaining bytes, let the bytes get dropped.
if pre_bs.len() > 0 {
let mut pre_reader = pre_bs.into_buf().reader();
let read_cnt = pre_reader.read(buf)?;
let mut new_pre = pre_reader.into_inner().into_inner();
new_pre.advance(read_cnt);
if prefix.len() > 0 {
let copy_len = cmp::min(prefix.len(), buf.len());
prefix.copy_to_slice(&mut buf[..copy_len]);
// Put back whats left
if new_pre.len() > 0 {
self.pre = Some(new_pre);
if prefix.len() > 0 {
self.pre = Some(prefix);
}
return Poll::Ready(Ok(read_cnt));
return Poll::Ready(Ok(copy_len));
}
}
Pin::new(&mut self.inner).poll_read(cx, buf)
@@ -118,7 +114,7 @@ mod tests {
// Rewind the stream so that it is as if we never read in the first place.
stream.rewind(Bytes::from(&buf[..]));
stream.rewind(Bytes::copy_from_slice(&buf[..]));
let mut buf = [0; 5];
stream
@@ -148,7 +144,7 @@ mod tests {
// Rewind the stream so that it is as if we never read in the first place.
stream.rewind(Bytes::from(&buf[..]));
stream.rewind(Bytes::copy_from_slice(&buf[..]));
let mut buf = [0; 5];
stream

View File

@@ -7,7 +7,6 @@ macro_rules! ready {
)
}
mod buf;
pub(crate) mod drain;
pub(crate) mod exec;
pub(crate) mod io;
@@ -15,8 +14,8 @@ mod lazy;
mod never;
pub(crate) mod task;
pub(crate) use self::buf::StaticBuf;
pub(crate) use self::exec::Exec;
pub(crate) use self::exec::{BoxSendFuture, Exec};
pub use self::exec::Executor;
pub(crate) use self::lazy::{lazy, Started as Lazy};
pub use self::never::Never;
pub(crate) use self::task::Poll;

View File

@@ -89,9 +89,6 @@ pub(crate) enum User {
/// User polled for an upgrade, but low-level API is not using upgrades.
ManualUpgrade,
/// Error trying to call `Executor::execute`.
Execute,
}
impl Error {
@@ -277,10 +274,6 @@ impl Error {
Error::new(Kind::Shutdown).with(cause)
}
pub(crate) fn new_execute<E: Into<Cause>>(cause: E) -> Error {
Error::new_user(User::Execute).with(cause)
}
pub(crate) fn new_h2(cause: ::h2::Error) -> Error {
if cause.is_io() {
Error::new_io(cause.into_io().expect("h2::Error::is_io"))
@@ -346,7 +339,6 @@ impl StdError for Error {
Kind::User(User::AbsoluteUriRequired) => "client requires absolute-form URIs",
Kind::User(User::NoUpgrade) => "no upgrade available",
Kind::User(User::ManualUpgrade) => "upgrade expected but low level API in use",
Kind::User(User::Execute) => "executor failed to spawn task",
}
}
@@ -398,12 +390,6 @@ impl From<http::uri::InvalidUri> for Parse {
}
}
impl From<http::uri::InvalidUriBytes> for Parse {
fn from(_: http::uri::InvalidUriBytes) -> Parse {
Parse::Uri
}
}
impl From<http::uri::InvalidUriParts> for Parse {
fn from(_: http::uri::InvalidUriParts) -> Parse {
Parse::Uri

View File

@@ -66,7 +66,6 @@ pub fn content_length_parse_all_values(values: ValueIter<'_, HeaderValue>) -> Op
pub fn set_content_length_if_missing(headers: &mut HeaderMap, len: u64) {
headers
.entry(CONTENT_LENGTH)
.unwrap()
.or_insert_with(|| HeaderValue::from(len));
}
@@ -105,7 +104,7 @@ pub fn add_chunked(mut entry: OccupiedEntry<'_, HeaderValue>) {
buf.copy_from_slice(b", ");
buf.copy_from_slice(CHUNKED.as_bytes());
*line = HeaderValue::from_shared(buf.freeze())
*line = HeaderValue::from_maybe_shared(buf.freeze())
.expect("original header value plus ascii is valid");
return;
}

View File

@@ -65,5 +65,5 @@ mod headers;
mod proto;
pub mod server;
pub mod service;
#[cfg(feature = "runtime")] pub mod rt;
pub mod rt;
pub mod upgrade;

View File

@@ -5,7 +5,7 @@ use std::marker::PhantomData;
use bytes::{Buf, Bytes};
use http::{HeaderMap, Method, Version};
use http::header::{HeaderValue, CONNECTION};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::Chunk;
use crate::common::{Pin, Poll, Unpin, task};
@@ -915,7 +915,11 @@ mod tests {
*conn.io.read_buf_mut() = ::bytes::BytesMut::from(&s[..]);
conn.state.cached_headers = Some(HeaderMap::with_capacity(2));
let mut rt = tokio::runtime::current_thread::Runtime::new().unwrap();
let mut rt = tokio::runtime::Builder::new()
.enable_all()
.basic_scheduler()
.build()
.unwrap();
b.iter(|| {
rt.block_on(futures_util::future::poll_fn(|cx| {

View File

@@ -328,7 +328,7 @@ impl StdError for IncompleteBody {
mod tests {
use std::time::Duration;
use std::pin::Pin;
use tokio_io::AsyncRead;
use tokio::io::AsyncRead;
use super::*;
impl<'a> MemRead for &'a [u8] {
@@ -336,7 +336,7 @@ mod tests {
let n = ::std::cmp::min(len, self.len());
if n > 0 {
let (a, b) = self.split_at(n);
let buf = Bytes::from(a);
let buf = Bytes::copy_from_slice(a);
*self = b;
Poll::Ready(Ok(buf))
} else {
@@ -349,7 +349,7 @@ mod tests {
fn read_mem(&mut self, cx: &mut task::Context<'_>, len: usize) -> Poll<io::Result<Bytes>> {
let mut v = vec![0; len];
let n = ready!(Pin::new(self).poll_read(cx, &mut v)?);
Poll::Ready(Ok(Bytes::from(&v[..n])))
Poll::Ready(Ok(Bytes::copy_from_slice(&v[..n])))
}
}

View File

@@ -2,7 +2,7 @@ use std::error::Error as StdError;
use bytes::{Buf, Bytes};
use http::{Request, Response, StatusCode};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::body::{Body, Payload};
use crate::common::{Future, Never, Poll, Pin, Unpin, task};
@@ -605,7 +605,7 @@ mod tests {
fn client_read_bytes_before_writing_request() {
let _ = pretty_env_logger::try_init();
tokio_test::task::mock(|cx| {
tokio_test::task::spawn(()).enter(|cx, _| {
let (io, mut handle) = tokio_test::io::Builder::new()
.build_with_handle();
@@ -637,12 +637,10 @@ mod tests {
});
}
#[test]
fn body_empty_chunks_ignored() {
#[tokio::test]
async fn body_empty_chunks_ignored() {
let _ = pretty_env_logger::try_init();
tokio_test::clock::mock(|_timer| {
tokio_test::task::mock(|cx| {
let io = tokio_test::io::Builder::new()
// no reading or writing, just be blocked for the test...
.wait(Duration::from_secs(5))
@@ -650,10 +648,10 @@ mod tests {
let (mut tx, rx) = crate::client::dispatch::channel();
let conn = Conn::<_, crate::Chunk, ClientTransaction>::new(io);
let mut dispatcher = Dispatcher::new(Client::new(rx), conn);
let mut dispatcher = tokio_test::task::spawn(Dispatcher::new(Client::new(rx), conn));
// First poll is needed to allow tx to send...
assert!(Pin::new(&mut dispatcher).poll(cx).is_pending());
assert!(dispatcher.poll().is_pending());
let body = {
let (mut tx, body) = crate::Body::channel();
@@ -665,8 +663,6 @@ mod tests {
// Ensure conn.write_body wasn't called with the empty chunk.
// If it is, it will trigger an assertion.
assert!(Pin::new(&mut dispatcher).poll(cx).is_pending());
});
});
assert!(dispatcher.poll().is_pending());
}
}

View File

@@ -1,12 +1,13 @@
use std::fmt;
use std::io::IoSlice;
use bytes::{Buf, IntoBuf};
use bytes::buf::{Chain, Take};
use iovec::IoVec;
use bytes::Buf;
use bytes::buf::ext::{BufExt, Chain, Take};
use crate::common::StaticBuf;
use super::io::WriteBuf;
type StaticBuf = &'static [u8];
/// Encoders to handle different Transfer-Encodings.
#[derive(Debug, Clone, PartialEq)]
pub struct Encoder {
@@ -84,17 +85,16 @@ impl Encoder {
match self.kind {
Kind::Length(0) => Ok(None),
Kind::Chunked => Ok(Some(EncodedBuf {
kind: BufKind::ChunkedEnd(StaticBuf(b"0\r\n\r\n")),
kind: BufKind::ChunkedEnd(b"0\r\n\r\n"),
})),
_ => Err(NotEof),
}
}
pub fn encode<B>(&mut self, msg: B) -> EncodedBuf<B::Buf>
pub fn encode<B>(&mut self, msg: B) -> EncodedBuf<B>
where
B: IntoBuf,
B: Buf,
{
let msg = msg.into_buf();
let len = msg.remaining();
debug_assert!(len > 0, "encode() called with empty buf");
@@ -103,7 +103,7 @@ impl Encoder {
trace!("encoding chunked {}B", len);
let buf = ChunkSize::new(len)
.chain(msg)
.chain(StaticBuf(b"\r\n"));
.chain(b"\r\n" as &'static [u8]);
BufKind::Chunked(buf)
},
Kind::Length(ref mut remaining) => {
@@ -127,11 +127,10 @@ impl Encoder {
}
}
pub(super) fn encode_and_end<B>(&self, msg: B, dst: &mut WriteBuf<EncodedBuf<B::Buf>>) -> bool
pub(super) fn encode_and_end<B>(&self, msg: B, dst: &mut WriteBuf<EncodedBuf<B>>) -> bool
where
B: IntoBuf,
B: Buf,
{
let msg = msg.into_buf();
let len = msg.remaining();
debug_assert!(len > 0, "encode() called with empty buf");
@@ -140,7 +139,7 @@ impl Encoder {
trace!("encoding chunked {}B", len);
let buf = ChunkSize::new(len)
.chain(msg)
.chain(StaticBuf(b"\r\n0\r\n\r\n"));
.chain(b"\r\n0\r\n\r\n" as &'static [u8]);
dst.buffer(buf);
!self.is_last
},
@@ -176,11 +175,10 @@ impl Encoder {
/// This is used in conjunction with Payload::__hyper_full_data(), which
/// means we can trust that the buf has the correct size (the buf itself
/// was checked to make the headers).
pub(super) fn danger_full_buf<B>(self, msg: B, dst: &mut WriteBuf<EncodedBuf<B::Buf>>)
pub(super) fn danger_full_buf<B>(self, msg: B, dst: &mut WriteBuf<EncodedBuf<B>>)
where
B: IntoBuf,
B: Buf,
{
let msg = msg.into_buf();
debug_assert!(msg.remaining() > 0, "encode() called with empty buf");
debug_assert!(match self.kind {
Kind::Length(len) => len == msg.remaining() as u64,
@@ -193,7 +191,7 @@ impl Encoder {
trace!("encoding chunked {}B", len);
let buf = ChunkSize::new(len)
.chain(msg)
.chain(StaticBuf(b"\r\n0\r\n\r\n"));
.chain(b"\r\n0\r\n\r\n" as &'static [u8]);
dst.buffer(buf);
},
_ => {
@@ -238,12 +236,12 @@ where
}
#[inline]
fn bytes_vec<'t>(&'t self, dst: &mut [&'t IoVec]) -> usize {
fn bytes_vectored<'t>(&'t self, dst: &mut [IoSlice<'t>]) -> usize {
match self.kind {
BufKind::Exact(ref b) => b.bytes_vec(dst),
BufKind::Limited(ref b) => b.bytes_vec(dst),
BufKind::Chunked(ref b) => b.bytes_vec(dst),
BufKind::ChunkedEnd(ref b) => b.bytes_vec(dst),
BufKind::Exact(ref b) => b.bytes_vectored(dst),
BufKind::Limited(ref b) => b.bytes_vectored(dst),
BufKind::Chunked(ref b) => b.bytes_vectored(dst),
BufKind::ChunkedEnd(ref b) => b.bytes_vectored(dst),
}
}
}

View File

@@ -2,11 +2,10 @@ use std::cell::Cell;
use std::cmp;
use std::collections::VecDeque;
use std::fmt;
use std::io;
use std::io::{self, IoSlice};
use bytes::{Buf, BufMut, Bytes, BytesMut};
use iovec::IoVec;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::common::{Pin, Poll, Unpin, task};
use super::{Http1Transaction, ParseContext, ParsedMessage};
@@ -105,6 +104,12 @@ where
&mut self.read_buf
}
/// Return the "allocated" available space, not the potential space
/// that could be allocated in the future.
fn read_buf_remaining_mut(&self) -> usize {
self.read_buf.capacity() - self.read_buf.len()
}
pub fn headers_buf(&mut self) -> &mut Vec<u8> {
let buf = self.write_buf.headers_mut();
&mut buf.bytes
@@ -170,7 +175,7 @@ where
pub fn poll_read_from_io(&mut self, cx: &mut task::Context<'_>) -> Poll<io::Result<usize>> {
self.read_blocked = false;
let next = self.read_buf_strategy.next();
if self.read_buf.remaining_mut() < next {
if self.read_buf_remaining_mut() < next {
self.read_buf.reserve(next);
}
match Pin::new(&mut self.io).poll_read_buf(cx, &mut self.read_buf) {
@@ -520,9 +525,9 @@ impl<B: Buf> Buf for WriteBuf<B> {
}
#[inline]
fn bytes_vec<'t>(&'t self, dst: &mut [&'t IoVec]) -> usize {
let n = self.headers.bytes_vec(dst);
self.queue.bytes_vec(&mut dst[n..]) + n
fn bytes_vectored<'t>(&'t self, dst: &mut [IoSlice<'t>]) -> usize {
let n = self.headers.bytes_vectored(dst);
self.queue.bytes_vectored(&mut dst[n..]) + n
}
}
@@ -562,9 +567,9 @@ impl<'a, B: Buf> Buf for WriteBufAuto<'a, B> {
}
#[inline]
fn bytes_vec<'t>(&'t self, dst: &mut [&'t IoVec]) -> usize {
fn bytes_vectored<'t>(&'t self, dst: &mut [IoSlice<'t>]) -> usize {
self.bytes_vec_called.set(true);
self.inner.bytes_vec(dst)
self.inner.bytes_vectored(dst)
}
}
@@ -638,13 +643,13 @@ impl<T: Buf> Buf for BufDeque<T> {
}
#[inline]
fn bytes_vec<'t>(&'t self, dst: &mut [&'t IoVec]) -> usize {
fn bytes_vectored<'t>(&'t self, dst: &mut [IoSlice<'t>]) -> usize {
if dst.is_empty() {
return 0;
}
let mut vecs = 0;
for buf in &self.bufs {
vecs += buf.bytes_vec(&mut dst[vecs..]);
vecs += buf.bytes_vectored(&mut dst[vecs..]);
if vecs == dst.len() {
break;
}

View File

@@ -23,7 +23,7 @@ macro_rules! header_name {
{
match HeaderName::from_bytes($bytes) {
Ok(name) => name,
Err(_) => panic!("illegal header name from httparse: {:?}", ::bytes::Bytes::from($bytes)),
Err(_) => panic!("illegal header name from httparse: {:?}", ::bytes::Bytes::copy_from_slice($bytes)),
}
}
@@ -40,7 +40,7 @@ macro_rules! header_value {
#[cfg(debug_assertions)]
{
let __hvb: ::bytes::Bytes = $bytes;
match HeaderValue::from_shared(__hvb.clone()) {
match HeaderValue::from_maybe_shared(__hvb.clone()) {
Ok(name) => name,
Err(_) => panic!("illegal header value from httparse: {:?}", __hvb),
}
@@ -50,7 +50,7 @@ macro_rules! header_value {
{
// Unsafe: httparse already validated header value
unsafe {
HeaderValue::from_shared_unchecked($bytes)
HeaderValue::from_maybe_shared_unchecked($bytes)
}
}
});
@@ -153,7 +153,7 @@ impl Http1Transaction for Server {
for header in &headers_indices[..headers_len] {
let name = header_name!(&slice[header.name.0..header.name.1]);
let value = header_value!(slice.slice(header.value.0, header.value.1));
let value = header_value!(slice.slice(header.value.0..header.value.1));
match name {
header::TRANSFER_ENCODING => {
@@ -302,10 +302,40 @@ impl Http1Transaction for Server {
let mut encoder = Encoder::length(0);
let mut wrote_date = false;
'headers: for (name, mut values) in msg.head.headers.drain() {
match name {
let mut cur_name = None;
let mut is_name_written = false;
let mut must_write_chunked = false;
let mut prev_con_len = None;
macro_rules! handle_is_name_written {
() => ({
if is_name_written {
// we need to clean up and write the newline
debug_assert_ne!(
&dst[dst.len() - 2 ..],
b"\r\n",
"previous header wrote newline but set is_name_written"
);
if must_write_chunked {
extend(dst, b", chunked\r\n");
} else {
extend(dst, b"\r\n");
}
}
})
}
'headers: for (opt_name, value) in msg.head.headers.drain() {
if let Some(n) = opt_name {
cur_name = Some(n);
handle_is_name_written!();
is_name_written = false;
}
let name = cur_name.as_ref().expect("current header name");
match *name {
header::CONTENT_LENGTH => {
if wrote_len {
if wrote_len && !is_name_written {
warn!("unexpected content-length found, canceling");
rewind(dst);
return Err(crate::Error::new_user_header());
@@ -319,78 +349,57 @@ impl Http1Transaction for Server {
//
// In debug builds, we'll assert they are the
// same to help developers find bugs.
encoder = Encoder::length(known_len);
#[cfg(debug_assertions)]
{
let mut folded = None::<(u64, HeaderValue)>;
for value in values {
if let Some(len) = headers::content_length_parse(&value) {
if let Some(fold) = folded {
if fold.0 != len {
panic!("multiple Content-Length values found: [{}, {}]", fold.0, len);
}
folded = Some(fold);
} else {
folded = Some((len, value));
}
} else {
panic!("illegal Content-Length value: {:?}", value);
}
}
if let Some((len, value)) = folded {
assert!(
len == known_len,
"payload claims content-length of {}, custom content-length header claims {}",
known_len,
len,
);
}
}
if !is_name_written {
encoder = Encoder::length(known_len);
extend(dst, b"content-length: ");
extend(dst, value.as_bytes());
extend(dst, b"\r\n");
wrote_len = true;
continue 'headers;
} else {
// No values in content-length... ignore?
continue 'headers;
}
is_name_written = true;
}
continue 'headers;
},
Some(BodyLength::Unknown) => {
// The Payload impl didn't know how long the
// body is, but a length header was included.
// We have to parse the value to return our
// Encoder...
let mut folded = None::<(u64, HeaderValue)>;
for value in values {
if let Some(len) = headers::content_length_parse(&value) {
if let Some(fold) = folded {
if fold.0 != len {
warn!("multiple Content-Length values found: [{}, {}]", fold.0, len);
if let Some(prev) = prev_con_len {
if prev != len {
warn!("multiple Content-Length values found: [{}, {}]", prev, len);
rewind(dst);
return Err(crate::Error::new_user_header());
}
folded = Some(fold);
debug_assert!(is_name_written);
continue 'headers;
} else {
folded = Some((len, value));
// we haven't written content-lenght yet!
encoder = Encoder::length(len);
extend(dst, b"content-length: ");
extend(dst, value.as_bytes());
wrote_len = true;
is_name_written = true;
prev_con_len = Some(len);
continue 'headers;
}
} else {
warn!("illegal Content-Length value: {:?}", value);
rewind(dst);
return Err(crate::Error::new_user_header());
}
}
if let Some((len, value)) = folded {
encoder = Encoder::length(len);
extend(dst, b"content-length: ");
extend(dst, value.as_bytes());
extend(dst, b"\r\n");
wrote_len = true;
continue 'headers;
} else {
// No values in content-length... ignore?
continue 'headers;
}
},
None => {
// We have no body to actually send,
@@ -402,11 +411,9 @@ impl Http1Transaction for Server {
if msg.req_method == &Some(Method::HEAD) {
debug_assert_eq!(encoder, Encoder::length(0));
} else {
for value in values {
if value.as_bytes() != b"0" {
warn!("content-length value found, but empty body provided: {:?}", value);
}
}
continue 'headers;
}
}
@@ -414,7 +421,7 @@ impl Http1Transaction for Server {
wrote_len = true;
},
header::TRANSFER_ENCODING => {
if wrote_len {
if wrote_len && !is_name_written {
warn!("unexpected transfer-encoding found, canceling");
rewind(dst);
return Err(crate::Error::new_user_header());
@@ -424,44 +431,36 @@ impl Http1Transaction for Server {
continue;
}
wrote_len = true;
// Must check each value, because `chunked` needs to be the
// last encoding, or else we add it.
must_write_chunked = !headers::is_chunked_(&value);
if !is_name_written {
encoder = Encoder::chunked();
is_name_written = true;
extend(dst, b"transfer-encoding: ");
let mut saw_chunked;
if let Some(te) = values.next() {
extend(dst, te.as_bytes());
saw_chunked = headers::is_chunked_(&te);
for value in values {
extend(dst, value.as_bytes());
} else {
extend(dst, b", ");
extend(dst, value.as_bytes());
saw_chunked = headers::is_chunked_(&value);
}
if !saw_chunked {
extend(dst, b", chunked\r\n");
} else {
extend(dst, b"\r\n");
}
} else {
// zero lines? add a chunked line then
extend(dst, b"chunked\r\n");
}
continue 'headers;
},
header::CONNECTION => {
if !is_last {
for value in values {
extend(dst, name.as_str().as_bytes());
extend(dst, b": ");
extend(dst, value.as_bytes());
extend(dst, b"\r\n");
if headers::connection_close(&value) {
is_last = true;
}
}
continue 'headers;
if !is_name_written {
is_name_written = true;
extend(dst, b"connection: ");
extend(dst, value.as_bytes());
} else {
extend(dst, b", ");
extend(dst, value.as_bytes());
}
continue 'headers;
},
header::DATE => {
wrote_date = true;
@@ -470,13 +469,20 @@ impl Http1Transaction for Server {
}
//TODO: this should perhaps instead combine them into
//single lines, as RFC7230 suggests is preferable.
for value in values {
// non-special write Name and Value
debug_assert!(
!is_name_written,
"{:?} set is_name_written and didn't continue loop",
name,
);
extend(dst, name.as_str().as_bytes());
extend(dst, b": ");
extend(dst, value.as_bytes());
extend(dst, b"\r\n");
}
}
handle_is_name_written!();
if !wrote_len {
encoder = match msg.body {
@@ -629,7 +635,7 @@ impl Http1Transaction for Client {
headers.reserve(headers_len);
for header in &headers_indices[..headers_len] {
let name = header_name!(&slice[header.name.0..header.name.1]);
let value = header_value!(slice.slice(header.value.0, header.value.1));
let value = header_value!(slice.slice(header.value.0..header.value.1));
if let header::CONNECTION = name {
// keep_alive was previously set to default for Version
@@ -820,8 +826,7 @@ impl Client {
// If the user set a transfer-encoding, respect that. Let's just
// make sure `chunked` is the final encoding.
let encoder = match headers.entry(header::TRANSFER_ENCODING)
.expect("TRANSFER_ENCODING is valid HeaderName") {
let encoder = match headers.entry(header::TRANSFER_ENCODING) {
Entry::Occupied(te) => {
should_remove_con_len = true;
if headers::is_chunked(te.iter()) {
@@ -906,8 +911,7 @@ fn set_content_length(headers: &mut HeaderMap, len: u64) -> Encoder {
// so perhaps only do that while the user is developing/testing.
if cfg!(debug_assertions) {
match headers.entry(header::CONTENT_LENGTH)
.expect("CONTENT_LENGTH is valid HeaderName") {
match headers.entry(header::CONTENT_LENGTH) {
Entry::Occupied(mut cl) => {
// Internal sanity check, we should have already determined
// that the header was illegal before calling this function.
@@ -1067,7 +1071,7 @@ mod tests {
#[test]
fn test_parse_request() {
let _ = pretty_env_logger::try_init();
let mut raw = BytesMut::from(b"GET /echo HTTP/1.1\r\nHost: hyper.rs\r\n\r\n".to_vec());
let mut raw = BytesMut::from("GET /echo HTTP/1.1\r\nHost: hyper.rs\r\n\r\n");
let mut method = None;
let msg = Server::parse(&mut raw, ParseContext {
cached_headers: &mut None,
@@ -1086,7 +1090,7 @@ mod tests {
#[test]
fn test_parse_response() {
let _ = pretty_env_logger::try_init();
let mut raw = BytesMut::from(b"HTTP/1.1 200 OK\r\nContent-Length: 0\r\n\r\n".to_vec());
let mut raw = BytesMut::from("HTTP/1.1 200 OK\r\nContent-Length: 0\r\n\r\n");
let ctx = ParseContext {
cached_headers: &mut None,
req_method: &mut Some(crate::Method::GET),
@@ -1101,7 +1105,7 @@ mod tests {
#[test]
fn test_parse_request_errors() {
let mut raw = BytesMut::from(b"GET htt:p// HTTP/1.1\r\nHost: hyper.rs\r\n\r\n".to_vec());
let mut raw = BytesMut::from("GET htt:p// HTTP/1.1\r\nHost: hyper.rs\r\n\r\n");
let ctx = ParseContext {
cached_headers: &mut None,
req_method: &mut None,
@@ -1480,7 +1484,7 @@ mod tests {
#[bench]
fn bench_parse_incoming(b: &mut Bencher) {
let mut raw = BytesMut::from(
b"GET /super_long_uri/and_whatever?what_should_we_talk_about/\
&b"GET /super_long_uri/and_whatever?what_should_we_talk_about/\
I_wonder/Hard_to_write_in_an_uri_after_all/you_have_to_make\
_up_the_punctuation_yourself/how_fun_is_that?test=foo&test1=\
foo1&test2=foo2&test3=foo3&test4=foo4 HTTP/1.1\r\nHost: \
@@ -1496,7 +1500,7 @@ mod tests {
X-Content-Duration: None\r\nX-Content-Security-Policy: None\
\r\nX-DNSPrefetch-Control: None\r\nX-Frame-Options: \
Something important obviously\r\nX-Requested-With: Nothing\
\r\n\r\n".to_vec()
\r\n\r\n"[..]
);
let len = raw.len();
let mut headers = Some(HeaderMap::new());
@@ -1526,7 +1530,7 @@ mod tests {
#[bench]
fn bench_parse_short(b: &mut Bencher) {
let s = &b"GET / HTTP/1.1\r\nHost: localhost:8080\r\n\r\n"[..];
let mut raw = BytesMut::from(s.to_vec());
let mut raw = BytesMut::from(s);
let len = raw.len();
let mut headers = Some(HeaderMap::new());

View File

@@ -2,7 +2,7 @@ use futures_channel::{mpsc, oneshot};
use futures_util::future::{self, FutureExt as _, TryFutureExt as _, Either};
use futures_util::stream::StreamExt as _;
use h2::client::{Builder, SendRequest};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::headers::content_length_parse_all;
use crate::body::Payload;
@@ -71,7 +71,7 @@ where
}
};
exec.execute(conn_task)?;
exec.execute(conn_task);
Ok(ClientTask {
conn_drop_ref,
@@ -155,7 +155,7 @@ where
drop(conn_drop_ref);
x
});
self.executor.execute(pipe)?;
self.executor.execute(pipe);
}
}
}
@@ -175,7 +175,7 @@ where
}
}
});
self.executor.execute(cb.send_when(fut))?;
self.executor.execute(cb.send_when(fut));
continue;
},

View File

@@ -4,7 +4,7 @@ use std::marker::Unpin;
use pin_project::{pin_project, project};
use h2::Reason;
use h2::server::{Builder, Connection, Handshake, SendResponse};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::body::Payload;
use crate::common::exec::H2Exec;
@@ -175,7 +175,7 @@ where
crate::Body::h2(stream, content_length)
});
let fut = H2Stream::new(service.call(req), respond);
exec.execute_h2stream(fut)?;
exec.execute_h2stream(fut);
},
Some(Err(e)) => {
return Poll::Ready(Err(crate::Error::new_h2(e)));
@@ -285,7 +285,6 @@ where
res
.headers_mut()
.entry(::http::header::DATE)
.expect("DATE is a valid HeaderName")
.or_insert_with(crate::proto::h1::date::update_and_header_value);

View File

@@ -1,42 +1,8 @@
//! Default runtime
//! Runtime components
//!
//! By default, hyper includes the [tokio](https://tokio.rs) runtime. To ease
//! using it, several types are re-exported here.
//! By default, hyper includes the [tokio](https://tokio.rs) runtime.
//!
//! The inclusion of a default runtime can be disabled by turning off hyper's
//! `runtime` Cargo feature.
//! If the `runtime` feature is disabled, the types in this module can be used
//! to plug in other runtimes.
pub use std::future::Future;
pub use futures_core::Stream;
use self::inner::Spawn;
/// Spawns a future on the default executor.
///
/// # Panics
///
/// This function will panic if the default executor is not set.
///
/// # Note
///
/// The `Spawn` return type is not currently meant for anything other than
/// to reserve adding new trait implementations to it later. It can be
/// ignored for now.
pub fn spawn<F>(f: F) -> Spawn
where
F: Future<Output = ()> + Send + 'static,
{
tokio::spawn(f);
Spawn {
_inner: (),
}
}
// Make the `Spawn` type an unnameable, so we can add
// methods or trait impls to it later without a breaking change.
mod inner {
#[allow(missing_debug_implementations)]
pub struct Spawn {
pub(super) _inner: (),
}
}
//pub use crate::common::Executor;

View File

@@ -12,13 +12,11 @@ use std::error::Error as StdError;
use std::fmt;
use std::mem;
#[cfg(feature = "tcp")] use std::net::SocketAddr;
#[cfg(feature = "tcp")] use std::time::Duration;
use bytes::Bytes;
use futures_core::Stream;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use pin_project::{pin_project, project};
#[cfg(feature = "tcp")] use tokio_net::driver::Handle;
use crate::body::{Body, Payload};
use crate::common::exec::{Exec, H2Exec, NewSvcExec};
@@ -357,7 +355,7 @@ impl<E> Http<E> {
/// # use hyper::{Body, Request, Response};
/// # use hyper::service::Service;
/// # use hyper::server::conn::Http;
/// # use tokio_io::{AsyncRead, AsyncWrite};
/// # use tokio::io::{AsyncRead, AsyncWrite};
/// # async fn run<I, S>(some_io: I, some_service: S)
/// # where
/// # I: AsyncRead + AsyncWrite + Unpin + Send + 'static,
@@ -419,74 +417,6 @@ impl<E> Http<E> {
}
}
#[cfg(feature = "tcp")]
#[doc(hidden)]
#[deprecated]
#[allow(deprecated)]
pub fn serve_addr<S, Bd>(&self, addr: &SocketAddr, make_service: S) -> crate::Result<Serve<AddrIncoming, S, E>>
where
S: MakeServiceRef<
AddrStream,
Body,
ResBody=Bd,
>,
S::Error: Into<Box<dyn StdError + Send + Sync>>,
S::Service: HttpService<Body>,
Bd: Payload,
E: H2Exec<<S::Service as HttpService<Body>>::Future, Bd>,
{
let mut incoming = AddrIncoming::new(addr, None)?;
if self.keep_alive {
incoming.set_keepalive(Some(Duration::from_secs(90)));
}
Ok(self.serve_incoming(incoming, make_service))
}
#[cfg(feature = "tcp")]
#[doc(hidden)]
#[deprecated]
#[allow(deprecated)]
pub fn serve_addr_handle<S, Bd>(&self, addr: &SocketAddr, handle: &Handle, make_service: S) -> crate::Result<Serve<AddrIncoming, S, E>>
where
S: MakeServiceRef<
AddrStream,
Body,
ResBody=Bd,
>,
S::Error: Into<Box<dyn StdError + Send + Sync>>,
Bd: Payload,
E: H2Exec<<S::Service as HttpService<Body>>::Future, Bd>,
{
let mut incoming = AddrIncoming::new(addr, Some(handle))?;
if self.keep_alive {
incoming.set_keepalive(Some(Duration::from_secs(90)));
}
Ok(self.serve_incoming(incoming, make_service))
}
#[doc(hidden)]
#[deprecated]
pub fn serve_incoming<I, IO, IE, S, Bd>(&self, incoming: I, make_service: S) -> Serve<I, S, E>
where
I: Accept<Conn=IO, Error=IE>,
IE: Into<Box<dyn StdError + Send + Sync>>,
IO: AsyncRead + AsyncWrite + Unpin,
S: MakeServiceRef<
IO,
Body,
ResBody=Bd,
>,
S::Error: Into<Box<dyn StdError + Send + Sync>>,
Bd: Payload,
E: H2Exec<<S::Service as HttpService<Body>>::Future, Bd>,
{
Serve {
incoming,
make_service,
protocol: self.clone(),
}
}
pub(super) fn serve<I, IO, IE, S, Bd>(&self, incoming: I, make_service: S) -> Serve<I, S, E>
where
I: Accept<Conn=IO, Error=IE>,
@@ -843,7 +773,7 @@ where
loop {
if let Some(connecting) = ready!(me.serve.as_mut().poll_next_(cx)?) {
let fut = NewSvcTask::new(connecting, watcher.clone());
me.serve.as_mut().project().protocol.exec.execute_new_svc(fut)?;
me.serve.as_mut().project().protocol.exec.execute_new_svc(fut);
} else {
return Poll::Ready(Ok(()));
}
@@ -876,7 +806,7 @@ where
pub(crate) mod spawn_all {
use std::error::Error as StdError;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::body::{Body, Payload};
use crate::common::exec::H2Exec;

View File

@@ -59,7 +59,7 @@ use std::fmt;
#[cfg(feature = "tcp")] use std::time::Duration;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use pin_project::pin_project;
use crate::body::{Body, Payload};
@@ -113,7 +113,7 @@ impl Server<AddrIncoming, ()> {
/// This method will panic if binding to the address fails. For a method
/// to bind to an address and return a `Result`, see `Server::try_bind`.
pub fn bind(addr: &SocketAddr) -> Builder<AddrIncoming> {
let incoming = AddrIncoming::new(addr, None)
let incoming = AddrIncoming::new(addr)
.unwrap_or_else(|e| {
panic!("error binding to {}: {}", addr, e);
});
@@ -122,14 +122,13 @@ impl Server<AddrIncoming, ()> {
/// Tries to bind to the provided address, and returns a [`Builder`](Builder).
pub fn try_bind(addr: &SocketAddr) -> crate::Result<Builder<AddrIncoming>> {
AddrIncoming::new(addr, None)
AddrIncoming::new(addr)
.map(Server::builder)
}
/// Create a new instance from a `std::net::TcpListener` instance.
pub fn from_tcp(listener: StdTcpListener) -> Result<Builder<AddrIncoming>, crate::Error> {
let handle = tokio_net::driver::Handle::default();
AddrIncoming::from_std(listener, &handle)
AddrIncoming::from_std(listener)
.map(Server::builder)
}
}

View File

@@ -1,6 +1,6 @@
use std::error::Error as StdError;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use pin_project::{pin_project, project};
use crate::body::{Body, Payload};

View File

@@ -4,9 +4,8 @@ use std::net::{SocketAddr, TcpListener as StdTcpListener};
use std::time::Duration;
use futures_util::FutureExt as _;
use tokio_net::driver::Handle;
use tokio_net::tcp::TcpListener;
use tokio_timer::Delay;
use tokio::net::TcpListener;
use tokio::time::Delay;
use crate::common::{Future, Pin, Poll, task};
@@ -25,20 +24,15 @@ pub struct AddrIncoming {
}
impl AddrIncoming {
pub(super) fn new(addr: &SocketAddr, handle: Option<&Handle>) -> crate::Result<Self> {
pub(super) fn new(addr: &SocketAddr) -> crate::Result<Self> {
let std_listener = StdTcpListener::bind(addr)
.map_err(crate::Error::new_listen)?;
if let Some(handle) = handle {
AddrIncoming::from_std(std_listener, handle)
} else {
let handle = Handle::default();
AddrIncoming::from_std(std_listener, &handle)
}
AddrIncoming::from_std(std_listener)
}
pub(super) fn from_std(std_listener: StdTcpListener, handle: &Handle) -> crate::Result<Self> {
let listener = TcpListener::from_std(std_listener, &handle)
pub(super) fn from_std(std_listener: StdTcpListener) -> crate::Result<Self> {
let listener = TcpListener::from_std(std_listener)
.map_err(crate::Error::new_listen)?;
let addr = listener.local_addr().map_err(crate::Error::new_listen)?;
Ok(AddrIncoming {
@@ -53,7 +47,7 @@ impl AddrIncoming {
/// Creates a new `AddrIncoming` binding to provided socket address.
pub fn bind(addr: &SocketAddr) -> crate::Result<Self> {
AddrIncoming::new(addr, None)
AddrIncoming::new(addr)
}
/// Get the local address bound to this listener.
@@ -135,7 +129,7 @@ impl AddrIncoming {
error!("accept error: {}", e);
// Sleep 1s.
let mut timeout = tokio_timer::delay_for(Duration::from_secs(1));
let mut timeout = tokio::time::delay_for(Duration::from_secs(1));
match Pin::new(&mut timeout).poll(cx) {
Poll::Ready(()) => {
@@ -197,8 +191,8 @@ mod addr_stream {
use std::io;
use std::net::SocketAddr;
use bytes::{Buf, BufMut};
use tokio_net::tcp::TcpStream;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::net::TcpStream;
use tokio::io::{AsyncRead, AsyncWrite};
use crate::common::{Pin, Poll, task};
@@ -232,8 +226,7 @@ mod addr_stream {
}
impl AsyncRead for AddrStream {
#[inline]
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool {
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [std::mem::MaybeUninit<u8>]) -> bool {
self.inner.prepare_uninitialized_buffer(buf)
}

View File

@@ -1,7 +1,7 @@
use std::error::Error as StdError;
use std::fmt;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::io::{AsyncRead, AsyncWrite};
use crate::body::Payload;
use crate::common::{Future, Poll, task};

View File

@@ -12,8 +12,8 @@ use std::io;
use std::marker::Unpin;
use bytes::{/*Buf, BufMut, */Bytes};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_sync::oneshot;
use tokio::io::{AsyncRead, AsyncWrite};
use tokio::sync::oneshot;
use crate::common::io::Rewind;
use crate::common::{Future, Pin, Poll, task};
@@ -136,7 +136,7 @@ impl Upgraded {
}
impl AsyncRead for Upgraded {
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool {
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [std::mem::MaybeUninit<u8>]) -> bool {
self.io.prepare_uninitialized_buffer(buf)
}

View File

@@ -15,9 +15,9 @@ use hyper::{Body, Client, Method, Request, StatusCode};
use futures_core::{Future, Stream, TryFuture};
use futures_channel::oneshot;
use futures_util::future::{self, FutureExt, TryFutureExt};
use futures_util::stream::TryStreamExt;
use tokio::runtime::current_thread::Runtime;
use tokio_net::tcp::TcpStream;
use futures_util::StreamExt;
use tokio::runtime::Runtime;
use tokio::net::TcpStream;
fn s(buf: &[u8]) -> &str {
::std::str::from_utf8(buf).expect("from_utf8")
@@ -27,6 +27,14 @@ fn tcp_connect(addr: &SocketAddr) -> impl Future<Output = std::io::Result<TcpStr
TcpStream::connect(*addr)
}
async fn concat(mut body: Body) -> Result<hyper::Chunk, hyper::Error> {
let mut vec = Vec::new();
while let Some(chunk) = body.next().await {
vec.extend_from_slice(&chunk?);
}
Ok(vec.into())
}
macro_rules! test {
(
name: $name:ident,
@@ -145,7 +153,7 @@ macro_rules! test {
);
)*
let body = rt.block_on(res.into_body().try_concat())
let body = rt.block_on(concat(res.into_body()))
.expect("body concat wait");
let expected_res_body = Option::<&[u8]>::from($response_body)
@@ -285,15 +293,15 @@ macro_rules! __client_req_prop {
});
($req_builder:ident, $body:ident, $addr:ident, method: $method:ident) => ({
$req_builder.method(Method::$method);
$req_builder = $req_builder.method(Method::$method);
});
($req_builder:ident, $body:ident, $addr:ident, version: $version:ident) => ({
$req_builder.version(hyper::Version::$version);
$req_builder = $req_builder.version(hyper::Version::$version);
});
($req_builder:ident, $body:ident, $addr:ident, url: $url:expr) => ({
$req_builder.uri(format!($url, addr=$addr));
$req_builder = $req_builder.uri(format!($url, addr=$addr));
});
($req_builder:ident, $body:ident, $addr:ident, body: $body_e:expr) => ({
@@ -304,7 +312,7 @@ macro_rules! __client_req_prop {
macro_rules! __client_req_header {
($req_builder:ident, { $($name:expr => $val:expr,)* }) => {
$(
$req_builder.header($name, $val);
$req_builder = $req_builder.header($name, $val);
)*
}
}
@@ -931,10 +939,10 @@ mod dispatch_impl {
use futures_core::{self, Future};
use futures_channel::{mpsc, oneshot};
use futures_util::future::{FutureExt, TryFutureExt};
use futures_util::stream::{StreamExt, TryStreamExt};
use tokio::runtime::current_thread::Runtime;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_net::tcp::TcpStream;
use futures_util::stream::{StreamExt};
use tokio::runtime::Runtime;
use tokio::io::{AsyncRead, AsyncWrite};
use tokio::net::TcpStream;
use hyper::client::connect::{Connected, Destination, HttpConnector};
use hyper::Client;
@@ -971,10 +979,13 @@ mod dispatch_impl {
.unwrap();
let res = client.request(req).map_ok(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
tokio_timer::delay_for(Duration::from_secs(1))
});
let rx = rx1.expect("thread panicked");
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
rt.block_on(async move {
let (res, ()) = future::join(res, rx).await;
res.unwrap();
tokio::time::delay_for(Duration::from_secs(1)).await;
});
rt.block_on(closes.into_future()).0.expect("closes");
}
@@ -1013,28 +1024,29 @@ mod dispatch_impl {
.unwrap();
client.request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
}).map_ok(|_| {
tokio_timer::delay_for(Duration::from_secs(1))
})
concat(res.into_body())
}).map_ok(|_| ())
};
// client is dropped
let rx = rx1.expect("thread panicked");
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
rt.block_on(async move {
let (res, ()) = future::join(res, rx).await;
res.unwrap();
tokio::time::delay_for(Duration::from_secs(1)).await;
});
rt.block_on(closes.into_future()).0.expect("closes");
}
#[test]
fn drop_client_closes_idle_connections() {
#[tokio::test]
async fn drop_client_closes_idle_connections() {
use futures_util::future;
let _ = pretty_env_logger::try_init();
let server = TcpListener::bind("127.0.0.1:0").unwrap();
let addr = server.local_addr().unwrap();
let mut rt = Runtime::new().unwrap();
let (closes_tx, mut closes) = mpsc::channel(10);
let (tx1, rx1) = oneshot::channel();
@@ -1065,35 +1077,37 @@ mod dispatch_impl {
.unwrap();
let res = client.request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
concat(res.into_body())
});
let rx = rx1.expect("thread panicked");
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
let (res, ()) = future::join(res, rx).await;
res.unwrap();
// not closed yet, just idle
{
rt.block_on(future::poll_fn(|ctx| {
future::poll_fn(|ctx| {
assert!(Pin::new(&mut closes).poll_next(ctx).is_pending());
Poll::Ready(Ok::<_, ()>(()))
})).unwrap();
}
Poll::Ready(())
}).await;
// drop to start the connections closing
drop(client);
let t = tokio_timer::delay_for(Duration::from_millis(100))
// and wait a few ticks for the connections to close
let t = tokio::time::delay_for(Duration::from_millis(100))
.map(|_| panic!("time out"));
let close = closes
.into_future()
.map(|(opt, _)| opt.expect("closes"));
let _ = rt.block_on(future::select(t, close));
future::select(t, close).await;
}
#[test]
fn drop_response_future_closes_in_progress_connection() {
#[tokio::test]
async fn drop_response_future_closes_in_progress_connection() {
let _ = pretty_env_logger::try_init();
let server = TcpListener::bind("127.0.0.1:0").unwrap();
let addr = server.local_addr().unwrap();
let mut rt = Runtime::new().unwrap();
let (closes_tx, closes) = mpsc::channel(10);
let (tx1, rx1) = oneshot::channel();
@@ -1122,27 +1136,28 @@ mod dispatch_impl {
.uri(&*format!("http://{}/a", addr))
.body(Body::empty())
.unwrap();
client.request(req)
client
.request(req)
.map(|_| unreachable!())
};
rt.block_on(future::select(res, rx1));
future::select(res, rx1).await;
// res now dropped
let t = tokio_timer::delay_for(Duration::from_millis(100))
let t = tokio::time::delay_for(Duration::from_millis(100))
.map(|_| panic!("time out"));
let close = closes
.into_future()
.map(|(opt, _)| opt.expect("closes"));
let _ = rt.block_on(future::select(t, close));
future::select(t, close).await;
}
#[test]
fn drop_response_body_closes_in_progress_connection() {
#[tokio::test]
async fn drop_response_body_closes_in_progress_connection() {
let _ = pretty_env_logger::try_init();
let server = TcpListener::bind("127.0.0.1:0").unwrap();
let addr = server.local_addr().unwrap();
let mut rt = Runtime::new().unwrap();
let (closes_tx, closes) = mpsc::channel(10);
let (tx1, rx1) = oneshot::channel();
@@ -1162,6 +1177,7 @@ mod dispatch_impl {
let _ = client_drop_rx.recv();
});
let rx = rx1.expect("thread panicked");
let res = {
let client = Client::builder()
.build(DebugConnector::with_http_and_closes(HttpConnector::new(), closes_tx));
@@ -1174,25 +1190,27 @@ mod dispatch_impl {
client.request(req)
};
let rx = rx1.expect("thread panicked");
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
let t = tokio_timer::delay_for(Duration::from_millis(100))
let (res, ()) = future::join(res, rx).await;
// drop the body
res.unwrap();
// and wait a few ticks to see the connection drop
let t = tokio::time::delay_for(Duration::from_millis(100))
.map(|_| panic!("time out"));
let close = closes
.into_future()
.map(|(opt, _)| opt.expect("closes"));
let _ = rt.block_on(future::select(t, close));
future::select(t, close).await;
}
#[test]
fn no_keep_alive_closes_connection() {
#[tokio::test]
async fn no_keep_alive_closes_connection() {
// https://github.com/hyperium/hyper/issues/1383
let _ = pretty_env_logger::try_init();
let server = TcpListener::bind("127.0.0.1:0").unwrap();
let addr = server.local_addr().unwrap();
let mut rt = Runtime::new().unwrap();
let (closes_tx, closes) = mpsc::channel(10);
let (tx1, rx1) = oneshot::channel();
@@ -1222,27 +1240,28 @@ mod dispatch_impl {
.unwrap();
let res = client.request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
concat(res.into_body())
});
let rx = rx1.expect("thread panicked");
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
let t = tokio_timer::delay_for(Duration::from_millis(100))
let (res, ()) = future::join(res, rx).await;
res.unwrap();
let t = tokio::time::delay_for(Duration::from_millis(100))
.map(|_| panic!("time out"));
let close = closes
.into_future()
.map(|(opt, _)| opt.expect("closes"));
let _ = rt.block_on(future::select(t, close));
future::select(t, close).await;
}
#[test]
fn socket_disconnect_closes_idle_conn() {
#[tokio::test]
async fn socket_disconnect_closes_idle_conn() {
// notably when keep-alive is enabled
let _ = pretty_env_logger::try_init();
let server = TcpListener::bind("127.0.0.1:0").unwrap();
let addr = server.local_addr().unwrap();
let mut rt = Runtime::new().unwrap();
let (closes_tx, closes) = mpsc::channel(10);
let (tx1, rx1) = oneshot::channel();
@@ -1266,17 +1285,19 @@ mod dispatch_impl {
.unwrap();
let res = client.request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
concat(res.into_body())
});
let rx = rx1.expect("thread panicked");
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
let t = tokio_timer::delay_for(Duration::from_millis(100))
let (res, ()) = future::join(res, rx).await;
res.unwrap();
let t = tokio::time::delay_for(Duration::from_millis(100))
.map(|_| panic!("time out"));
let close = closes
.into_future()
.map(|(opt, _)| opt.expect("closes"));
let _ = rt.block_on(future::select(t, close));
future::select(t, close).await;
}
#[test]
@@ -1466,7 +1487,7 @@ mod dispatch_impl {
assert_eq!(connects.load(Ordering::Relaxed), 0);
let delayed_body = rx1
.then(|_| tokio_timer::delay_for(Duration::from_millis(200)))
.then(|_| tokio::time::delay_for(Duration::from_millis(200)))
.map(|_| Ok::<_, ()>("hello a"))
.map_err(|_| -> hyper::Error { panic!("rx1") })
.into_stream();
@@ -1481,7 +1502,7 @@ mod dispatch_impl {
// req 1
let fut = future::join(client.request(req), rx)
.then(|_| tokio_timer::delay_for(Duration::from_millis(200)))
.then(|_| tokio::time::delay_for(Duration::from_millis(200)))
// req 2
.then(move |()| {
let rx = rx3.expect("thread panicked");
@@ -1639,11 +1660,11 @@ mod dispatch_impl {
use hyper::Response;
use hyper::server::conn::Http;
use hyper::service::service_fn;
use tokio_net::tcp::TcpListener;
use tokio::net::TcpListener;
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let mut listener = rt.block_on(TcpListener::bind("127.0.0.1:0")).unwrap();
let mut listener = rt.block_on(TcpListener::bind(SocketAddr::from(([127, 0, 0, 1], 0)))).unwrap();
let addr = listener.local_addr().unwrap();
let mut connector = DebugConnector::new();
connector.alpn_h2 = true;
@@ -1782,7 +1803,7 @@ mod dispatch_impl {
mod conn {
use std::io::{self, Read, Write};
use std::net::TcpListener;
use std::net::{SocketAddr, TcpListener};
use std::pin::Pin;
use std::task::{Context, Poll};
use std::thread;
@@ -1790,20 +1811,20 @@ mod conn {
use futures_channel::oneshot;
use futures_util::future::{self, poll_fn, FutureExt, TryFutureExt};
use futures_util::stream::TryStreamExt;
use tokio::runtime::current_thread::Runtime;
use tokio_io::{AsyncRead, AsyncReadExt as _, AsyncWrite, AsyncWriteExt as _};
use tokio_net::tcp::{TcpListener as TkTcpListener, TcpStream};
use futures_util::StreamExt;
use tokio::runtime::Runtime;
use tokio::io::{AsyncRead, AsyncReadExt as _, AsyncWrite, AsyncWriteExt as _};
use tokio::net::{TcpListener as TkTcpListener, TcpStream};
use hyper::{self, Request, Body, Method};
use hyper::client::conn;
use super::{s, tcp_connect, FutureHyperExt};
use super::{concat, s, tcp_connect, FutureHyperExt};
#[tokio::test]
async fn get() {
let _ = ::pretty_env_logger::try_init();
let mut listener = TkTcpListener::bind("127.0.0.1:0").await.unwrap();
let mut listener = TkTcpListener::bind(SocketAddr::from(([127, 0, 0, 1], 0))).await.unwrap();
let addr = listener.local_addr().unwrap();
let server = async move {
@@ -1824,7 +1845,7 @@ mod conn {
let tcp = tcp_connect(&addr).await.expect("connect");
let (mut client, conn) = conn::handshake(tcp).await.expect("handshake");
hyper::rt::spawn(async move {
tokio::task::spawn(async move {
conn.await.expect("http conn");
});
@@ -1842,7 +1863,7 @@ mod conn {
#[test]
fn incoming_content_length() {
use hyper::body::Payload;
use hyper::body::HttpBody;
let server = TcpListener::bind("127.0.0.1:0").unwrap();
let addr = server.local_addr().unwrap();
@@ -1882,7 +1903,7 @@ mod conn {
});
let rx = rx1.expect("thread panicked");
let rx = rx.then(|_| tokio_timer::delay_for(Duration::from_millis(200)));
let rx = rx.then(|_| tokio::time::delay_for(Duration::from_millis(200)));
let chunk = rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
assert_eq!(chunk.len(), 5);
}
@@ -1971,10 +1992,10 @@ mod conn {
let res = client.send_request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
concat(res.into_body())
});
let rx = rx1.expect("thread panicked");
let rx = rx.then(|_| tokio_timer::delay_for(Duration::from_millis(200)));
let rx = rx.then(|_| tokio::time::delay_for(Duration::from_millis(200)));
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
}
@@ -2015,10 +2036,10 @@ mod conn {
let res = client.send_request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
concat(res.into_body())
});
let rx = rx1.expect("thread panicked");
let rx = rx.then(|_| tokio_timer::delay_for(Duration::from_millis(200)));
let rx = rx.then(|_| tokio::time::delay_for(Duration::from_millis(200)));
rt.block_on(future::join(res, rx).map(|r| r.0)).unwrap();
}
@@ -2053,7 +2074,7 @@ mod conn {
.unwrap();
let res1 = client.send_request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
concat(res.into_body())
});
// pipelined request will hit NotReady, and thus should return an Error::Cancel
@@ -2070,7 +2091,7 @@ mod conn {
});
let rx = rx1.expect("thread panicked");
let rx = rx.then(|_| tokio_timer::delay_for(Duration::from_millis(200)));
let rx = rx.then(|_| tokio::time::delay_for(Duration::from_millis(200)));
rt.block_on(future::join3(res1, res2, rx).map(|r| r.0)).unwrap();
}
@@ -2126,11 +2147,11 @@ mod conn {
let res = client.send_request(req).and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::SWITCHING_PROTOCOLS);
assert_eq!(res.headers()["Upgrade"], "foobar");
res.into_body().try_concat()
concat(res.into_body())
});
let rx = rx1.expect("thread panicked");
let rx = rx.then(|_| tokio_timer::delay_for(Duration::from_millis(200)));
let rx = rx.then(|_| tokio::time::delay_for(Duration::from_millis(200)));
rt.block_on(future::join3(until_upgrade, res, rx).map(|r| r.0)).unwrap();
// should not be ready now
@@ -2211,14 +2232,14 @@ mod conn {
.send_request(req)
.and_then(move |res| {
assert_eq!(res.status(), hyper::StatusCode::OK);
res.into_body().try_concat()
concat(res.into_body())
})
.map_ok(|body| {
assert_eq!(body.as_ref(), b"");
});
let rx = rx1.expect("thread panicked");
let rx = rx.then(|_| tokio_timer::delay_for(Duration::from_millis(200)));
let rx = rx.then(|_| tokio::time::delay_for(Duration::from_millis(200)));
rt.block_on(future::join3(until_tunneled, res, rx).map(|r| r.0)).unwrap();
// should not be ready now
@@ -2247,16 +2268,14 @@ mod conn {
assert_eq!(vec, b"bar=foo");
}
#[test]
fn http2_detect_conn_eof() {
#[tokio::test]
async fn http2_detect_conn_eof() {
use futures_util::future;
use hyper::{Response, Server};
use hyper::service::{make_service_fn, service_fn};
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let server = Server::bind(&([127, 0, 0, 1], 0).into())
.http2_only(true)
.serve(make_service_fn(|_| async move {
@@ -2264,40 +2283,53 @@ mod conn {
}));
let addr = server.local_addr();
let (shdn_tx, shdn_rx) = oneshot::channel();
rt.spawn(server.with_graceful_shutdown(async {
shdn_rx.await.ok();
}).map(|_| ()));
tokio::task::spawn(async move {
server
.with_graceful_shutdown(async move {
let _ = shdn_rx.await;
})
.await
.expect("server")
});
let io = rt.block_on(tcp_connect(&addr)).expect("tcp connect");
let (mut client, conn) = rt.block_on(
conn::Builder::new().http2_only(true).handshake::<_, Body>(io)
).expect("http handshake");
let io = tcp_connect(&addr).await.expect("tcp connect");
let (mut client, conn) = conn::Builder::new()
.http2_only(true)
.handshake::<_, Body>(io)
.await
.expect("http handshake");
rt.spawn(conn
.map_err(|e| panic!("client conn error: {:?}", e))
.map(|_| ()));
tokio::task::spawn(async move {
conn.await.expect("client conn");
});
// Sanity check that client is ready
rt.block_on(future::poll_fn(|ctx| client.poll_ready(ctx))).expect("client poll ready sanity");
future::poll_fn(|ctx| client.poll_ready(ctx))
.await
.expect("client poll ready sanity");
let req = Request::builder()
.uri(format!("http://{}/", addr))
.body(Body::empty())
.expect("request builder");
rt.block_on(client.send_request(req)).expect("req1 send");
client.send_request(req).await.expect("req1 send");
// Sanity check that client is STILL ready
rt.block_on(future::poll_fn(|ctx| client.poll_ready(ctx))).expect("client poll ready after");
future::poll_fn(|ctx| client.poll_ready(ctx))
.await
.expect("client poll ready after");
// Trigger the server shutdown...
let _ = shdn_tx.send(());
// Allow time for graceful shutdown roundtrips...
rt.block_on(tokio_timer::delay_for(Duration::from_millis(100)));
tokio::time::delay_for(Duration::from_millis(100)).await;
// After graceful shutdown roundtrips, the client should be closed...
rt.block_on(future::poll_fn(|ctx| client.poll_ready(ctx))).expect_err("client should be closed");
future::poll_fn(|ctx| client.poll_ready(ctx))
.await
.expect_err("client should be closed");
}
struct DebugStream {

View File

@@ -1,6 +1,7 @@
#![deny(warnings)]
#![deny(rust_2018_idioms)]
use std::future::Future;
use std::net::{TcpStream, Shutdown, SocketAddr};
use std::io::{self, Read, Write};
use std::sync::atomic::{AtomicBool, Ordering};
@@ -13,18 +14,13 @@ use std::thread;
use std::time::Duration;
use futures_channel::oneshot;
use futures_core::ready;
use futures_core::future::BoxFuture;
use futures_util::future::{self, Either, FutureExt, TryFutureExt};
#[cfg(feature = "unstable-stream")]
use futures_util::stream::StreamExt as _;
// TODO: remove once tokio is updated to futures 0.3
use futures_util_a19::stream::StreamExt as _;
use http::header::{HeaderName, HeaderValue};
use tokio_net::driver::Handle;
use tokio_net::tcp::{TcpListener, TcpStream as TkTcpStream};
use tokio::runtime::current_thread::Runtime;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio::net::{TcpListener, TcpStream as TkTcpStream};
use tokio::runtime::Runtime;
use tokio::io::{AsyncRead, AsyncWrite};
use hyper::{Body, Request, Response, StatusCode, Version};
use hyper::client::Client;
@@ -120,19 +116,22 @@ mod response_body_lengths {
assert_eq!(
case.expects_chunked,
has_header(&body, "transfer-encoding:"),
"expects_chunked"
"expects_chunked, headers = {:?}",
body
);
assert_eq!(
case.expects_chunked,
has_header(&body, "chunked\r\n"),
"expects_chunked"
"expects_chunked, headers = {:?}",
body
);
assert_eq!(
case.expects_con_len,
has_header(&body, "content-length:"),
"expects_con_len"
"expects_con_len, headers = {:?}",
body
);
let n = body.find("\r\n\r\n").unwrap() + 4;
@@ -295,16 +294,14 @@ mod response_body_lengths {
});
}
#[test]
fn http2_auto_response_with_known_length() {
#[tokio::test]
async fn http2_auto_response_with_known_length() {
use http_body::Body;
let server = serve();
let addr_str = format!("http://{}", server.addr());
server.reply().body("Hello, World!");
let mut rt = Runtime::new().expect("rt new");
rt.block_on({
let client = Client::builder()
.http2_only(true)
.build_http::<hyper::Body>();
@@ -312,19 +309,13 @@ mod response_body_lengths {
.parse::<hyper::Uri>()
.expect("server addr should parse");
client
.get(uri)
.map_ok(|res| {
let res = client.get(uri).await.unwrap();
assert_eq!(res.headers().get("content-length").unwrap(), "13");
assert_eq!(res.body().size_hint().exact(), Some(13));
()
})
.map_err(|_e| ())
}).unwrap();
}
#[test]
fn http2_auto_response_with_conflicting_lengths() {
#[tokio::test]
async fn http2_auto_response_with_conflicting_lengths() {
use http_body::Body;
let server = serve();
@@ -334,8 +325,6 @@ mod response_body_lengths {
.header("content-length", "10")
.body("Hello, World!");
let mut rt = Runtime::new().expect("rt new");
rt.block_on({
let client = Client::builder()
.http2_only(true)
.build_http::<hyper::Body>();
@@ -343,15 +332,9 @@ mod response_body_lengths {
.parse::<hyper::Uri>()
.expect("server addr should parse");
client
.get(uri)
.map_ok(|res| {
let res = client.get(uri).await.unwrap();
assert_eq!(res.headers().get("content-length").unwrap(), "10");
assert_eq!(res.body().size_hint().exact(), Some(10));
()
})
.map_err(|_e| ())
}).unwrap();
}
}
@@ -855,10 +838,9 @@ fn http_10_request_receives_http_10_response() {
assert_eq!(s(&buf[..expected.len()]), expected);
}
#[test]
fn disable_keep_alive_mid_request() {
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
#[tokio::test]
async fn disable_keep_alive_mid_request() {
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
let (tx1, rx1) = oneshot::channel();
@@ -874,11 +856,7 @@ fn disable_keep_alive_mid_request() {
req.read_to_end(&mut buf).unwrap();
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
let srv = Http::new().serve_connection(socket, HelloWorld);
future::try_select(srv, rx1)
.then(|r| {
@@ -893,17 +871,16 @@ fn disable_keep_alive_mid_request() {
Err(Either::Right((e, _))) => panic!("unexpected error {}", e),
}
})
});
.await
.unwrap();
rt.block_on(fut).unwrap();
child.join().unwrap();
}
#[test]
fn disable_keep_alive_post_request() {
#[tokio::test]
async fn disable_keep_alive_post_request() {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
let (tx1, rx1) = oneshot::channel();
@@ -929,17 +906,13 @@ fn disable_keep_alive_post_request() {
let dropped = Dropped::new();
let dropped2 = dropped.clone();
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
let transport = DebugStream {
stream: socket,
_debug: dropped2,
};
let server = Http::new().serve_connection(transport, HelloWorld);
future::try_select(server, rx1)
let fut = future::try_select(server, rx1)
.then(|r| {
match r {
Ok(Either::Left(_)) => panic!("expected rx first"),
@@ -950,45 +923,34 @@ fn disable_keep_alive_post_request() {
Err(Either::Left((e, _))) => panic!("unexpected error {}", e),
Err(Either::Right((e, _))) => panic!("unexpected error {}", e),
}
})
});
assert!(!dropped.load());
rt.block_on(fut).unwrap();
// we must poll the Core one more time in order for Windows to drop
// the read-blocked socket.
//
// See https://github.com/carllerche/mio/issues/776
let timeout = tokio_timer::delay_for(Duration::from_millis(10));
rt.block_on(timeout);
fut.await.unwrap();
assert!(dropped.load());
child.join().unwrap();
}
#[test]
fn empty_parse_eof_does_not_return_error() {
#[tokio::test]
async fn empty_parse_eof_does_not_return_error() {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
thread::spawn(move || {
let _tcp = connect(&addr);
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| Http::new().serve_connection(socket, HelloWorld));
rt.block_on(fut).expect("empty parse eof is ok");
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.serve_connection(socket, HelloWorld)
.await
.expect("empty parse eof is ok");
}
#[test]
fn nonempty_parse_eof_returns_error() {
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
#[tokio::test]
async fn nonempty_parse_eof_returns_error() {
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
thread::spawn(move || {
@@ -996,20 +958,17 @@ fn nonempty_parse_eof_returns_error() {
tcp.write_all(b"GET / HTTP/1.1").unwrap();
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| Http::new().serve_connection(socket, HelloWorld));
rt.block_on(fut).expect_err("partial parse eof is error");
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.serve_connection(socket, HelloWorld)
.await
.expect_err("partial parse eof is error");
}
#[test]
fn http1_allow_half_close() {
#[tokio::test]
async fn http1_allow_half_close() {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
let t1 = thread::spawn(move || {
@@ -1023,28 +982,23 @@ fn http1_allow_half_close() {
assert_eq!(s(&buf[..expected.len()]), expected);
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.http1_half_close(true)
.serve_connection(socket, service_fn(|_| {
tokio_timer::delay_for(Duration::from_millis(500))
tokio::time::delay_for(Duration::from_millis(500))
.map(|_| Ok::<_, hyper::Error>(Response::new(Body::empty())))
}))
});
.await
.unwrap();
rt.block_on(fut).unwrap();
t1.join().expect("client thread");
}
#[test]
fn disconnect_after_reading_request_before_responding() {
#[tokio::test]
async fn disconnect_after_reading_request_before_responding() {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
thread::spawn(move || {
@@ -1052,28 +1006,22 @@ fn disconnect_after_reading_request_before_responding() {
tcp.write_all(b"GET / HTTP/1.1\r\n\r\n").unwrap();
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.http1_half_close(false)
.serve_connection(socket, service_fn(|_| {
tokio_timer::delay_for(Duration::from_secs(2))
tokio::time::delay_for(Duration::from_secs(2))
.map(|_| -> Result<Response<Body>, hyper::Error> {
panic!("response future should have been dropped");
})
}))
});
rt.block_on(fut).expect_err("socket disconnected");
.await
.expect_err("socket disconnected");
}
#[test]
fn returning_1xx_response_is_error() {
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
#[tokio::test]
async fn returning_1xx_response_is_error() {
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
thread::spawn(move || {
@@ -1086,11 +1034,7 @@ fn returning_1xx_response_is_error() {
assert_eq!(s(&buf[..expected.len()]), expected);
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.serve_connection(socket, service_fn(|_| async move {
Ok::<_, hyper::Error>(Response::builder()
@@ -1098,9 +1042,8 @@ fn returning_1xx_response_is_error() {
.body(Body::empty())
.unwrap())
}))
});
rt.block_on(fut).expect_err("1xx status code should error");
.await
.expect_err("1xx status code should error");
}
#[test]
@@ -1121,13 +1064,12 @@ fn header_name_too_long() {
assert!(s(&buf[..n]).starts_with("HTTP/1.1 431 Request Header Fields Too Large\r\n"));
}
#[test]
fn upgrades() {
#[tokio::test]
async fn upgrades() {
use tokio::io::{AsyncReadExt, AsyncWriteExt};
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
let (tx, rx) = oneshot::channel();
@@ -1152,11 +1094,7 @@ fn upgrades() {
tcp.write_all(b"bar=foo").expect("write 2");
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
let conn = Http::new()
.serve_connection(socket, service_fn(|_| {
let res = Response::builder()
@@ -1167,36 +1105,25 @@ fn upgrades() {
future::ready(Ok::<_, hyper::Error>(res))
}));
let mut conn_opt = Some(conn);
future::poll_fn(move |ctx| {
ready!(conn_opt.as_mut().unwrap().poll_without_shutdown(ctx)).unwrap();
// conn is done with HTTP now
Poll::Ready(Ok(conn_opt.take().unwrap()))
})
});
let conn = rt.block_on(fut).unwrap();
// wait so that we don't write until other side saw 101 response
rt.block_on(rx).unwrap();
let parts = conn.into_parts();
let parts = conn.without_shutdown().await.unwrap();
assert_eq!(parts.read_buf, "eagerly optimistic");
// wait so that we don't write until other side saw 101 response
rx.await.unwrap();
let mut io = parts.io;
rt.block_on(io.write_all(b"foo=bar")).unwrap();
io.write_all(b"foo=bar").await.unwrap();
let mut vec = vec![];
rt.block_on(io.read_to_end(&mut vec)).unwrap();
io.read_to_end(&mut vec).await.unwrap();
assert_eq!(vec, b"bar=foo");
}
#[test]
fn http_connect() {
#[tokio::test]
async fn http_connect() {
use tokio::io::{AsyncReadExt, AsyncWriteExt};
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
let (tx, rx) = oneshot::channel();
@@ -1219,11 +1146,7 @@ fn http_connect() {
tcp.write_all(b"bar=foo").expect("write 2");
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
let conn = Http::new()
.serve_connection(socket, service_fn(|_| {
let res = Response::builder()
@@ -1233,36 +1156,25 @@ fn http_connect() {
future::ready(Ok::<_, hyper::Error>(res))
}));
let mut conn_opt = Some(conn);
future::poll_fn(move |ctx| {
ready!(conn_opt.as_mut().unwrap().poll_without_shutdown(ctx)).unwrap();
// conn is done with HTTP now
Poll::Ready(Ok(conn_opt.take().unwrap()))
})
});
let conn = rt.block_on(fut).unwrap();
// wait so that we don't write until other side saw 101 response
rt.block_on(rx).unwrap();
let parts = conn.into_parts();
let parts = conn.without_shutdown().await.unwrap();
assert_eq!(parts.read_buf, "eagerly optimistic");
// wait so that we don't write until other side saw 101 response
rx.await.unwrap();
let mut io = parts.io;
rt.block_on(io.write_all(b"foo=bar")).unwrap();
io.write_all(b"foo=bar").await.unwrap();
let mut vec = vec![];
rt.block_on(io.read_to_end(&mut vec)).unwrap();
io.read_to_end(&mut vec).await.unwrap();
assert_eq!(vec, b"bar=foo");
}
#[test]
fn upgrades_new() {
#[tokio::test]
async fn upgrades_new() {
use tokio::io::{AsyncReadExt, AsyncWriteExt};
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
let (read_101_tx, read_101_rx) = oneshot::channel();
@@ -1300,38 +1212,35 @@ fn upgrades_new() {
.unwrap())
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
Http::new().serve_connection(socket, svc).with_upgrades()
});
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.serve_connection(socket, svc)
.with_upgrades()
.await
.unwrap();
rt.block_on(fut).unwrap();
let on_upgrade = upgrades_rx.recv().unwrap();
// wait so that we don't write until other side saw 101 response
rt.block_on(read_101_rx).unwrap();
read_101_rx.await.unwrap();
let upgraded = rt.block_on(on_upgrade).unwrap();
let upgraded = on_upgrade.await.expect("on_upgrade");
let parts = upgraded.downcast::<TkTcpStream>().unwrap();
assert_eq!(parts.read_buf, "eagerly optimistic");
let mut io = parts.io;
rt.block_on(io.write_all(b"foo=bar")).unwrap();
io.write_all(b"foo=bar").await.unwrap();
let mut vec = vec![];
rt.block_on(io.read_to_end(&mut vec)).unwrap();
io.read_to_end(&mut vec).await.unwrap();
assert_eq!(s(&vec), "bar=foo");
}
#[test]
fn http_connect_new() {
#[tokio::test]
async fn http_connect_new() {
use tokio::io::{AsyncReadExt, AsyncWriteExt};
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
let (read_200_tx, read_200_rx) = oneshot::channel();
@@ -1366,38 +1275,33 @@ fn http_connect_new() {
.unwrap())
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
Http::new().serve_connection(socket, svc).with_upgrades()
});
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.serve_connection(socket, svc)
.with_upgrades()
.await
.unwrap();
rt.block_on(fut).unwrap();
let on_upgrade = upgrades_rx.recv().unwrap();
// wait so that we don't write until other side saw 200
rt.block_on(read_200_rx).unwrap();
read_200_rx.await.unwrap();
let upgraded = rt.block_on(on_upgrade).unwrap();
let upgraded = on_upgrade.await.expect("on_upgrade");
let parts = upgraded.downcast::<TkTcpStream>().unwrap();
assert_eq!(parts.read_buf, "eagerly optimistic");
let mut io = parts.io;
rt.block_on(io.write_all(b"foo=bar")).unwrap();
io.write_all(b"foo=bar").await.unwrap();
let mut vec = vec![];
rt.block_on(io.read_to_end(&mut vec)).unwrap();
io.read_to_end(&mut vec).await.unwrap();
assert_eq!(s(&vec), "bar=foo");
}
#[test]
fn parse_errors_send_4xx_response() {
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
#[tokio::test]
async fn parse_errors_send_4xx_response() {
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
thread::spawn(move || {
@@ -1410,21 +1314,16 @@ fn parse_errors_send_4xx_response() {
assert_eq!(s(&buf[..expected.len()]), expected);
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| Http::new().serve_connection(socket, HelloWorld));
rt.block_on(fut).expect_err("HTTP parse error");
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.serve_connection(socket, HelloWorld)
.await
.expect_err("HTTP parse error");
}
#[test]
fn illegal_request_length_returns_400_response() {
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
#[tokio::test]
async fn illegal_request_length_returns_400_response() {
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
thread::spawn(move || {
@@ -1437,13 +1336,11 @@ fn illegal_request_length_returns_400_response() {
assert_eq!(s(&buf[..expected.len()]), expected);
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| Http::new().serve_connection(socket, HelloWorld));
rt.block_on(fut).expect_err("illegal Content-Length should error");
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.serve_connection(socket, HelloWorld)
.await
.expect_err("illegal Content-Length should error");
}
#[test]
@@ -1458,11 +1355,10 @@ fn max_buf_size_no_panic() {
Http::new().max_buf_size(MAX);
}
#[test]
fn max_buf_size() {
#[tokio::test]
async fn max_buf_size() {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().unwrap();
let listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let mut listener = tcp_bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let addr = listener.local_addr().unwrap();
const MAX: usize = 16_000;
@@ -1478,17 +1374,12 @@ fn max_buf_size() {
assert_eq!(s(&buf[..expected.len()]), expected);
});
let mut incoming = listener.incoming();
let fut = incoming.next()
.map(Option::unwrap)
.map_err(|_| unreachable!())
.and_then(|socket| {
let (socket, _) = listener.accept().await.unwrap();
Http::new()
.max_buf_size(MAX)
.serve_connection(socket, HelloWorld)
});
rt.block_on(fut).expect_err("should TooLarge error");
.await
.expect_err("should TooLarge error");
}
#[cfg(feature = "unstable-stream")]
@@ -1576,8 +1467,8 @@ fn http1_only() {
}).unwrap_err();
}
#[test]
fn http2_service_error_sends_reset_reason() {
#[tokio::test]
async fn http2_service_error_sends_reset_reason() {
use std::error::Error;
let server = serve();
@@ -1587,22 +1478,20 @@ fn http2_service_error_sends_reset_reason() {
.reply()
.error(h2::Error::from(h2::Reason::INADEQUATE_SECURITY));
let mut rt = Runtime::new().expect("runtime new");
let err = rt.block_on({
let client = Client::builder()
.http2_only(true)
.build_http::<hyper::Body>();
let uri = addr_str.parse().expect("server addr should parse");
client.get(uri)
}).unwrap_err();
dbg!("start");
let err = dbg!(Client::builder()
.http2_only(true)
.build_http::<hyper::Body>()
.get(uri)
.await
.expect_err("client.get"));
let h2_err = err
.source()
.unwrap()
.expect("err.source")
.downcast_ref::<h2::Error>()
.unwrap();
.expect("downcast");
assert_eq!(h2_err.reason(), Some(h2::Reason::INADEQUATE_SECURITY));
}
@@ -1648,9 +1537,9 @@ fn http2_body_user_error_sends_reset_reason() {
assert_eq!(h2_err.reason(), Some(h2::Reason::INADEQUATE_SECURITY));
}
struct Svc;
struct Http2ReadyErrorSvc;
impl tower_service::Service<Request<Body>> for Svc {
impl tower_service::Service<Request<Body>> for Http2ReadyErrorSvc {
type Response = Response<Body>;
type Error = h2::Error;
type Future = Box<dyn futures_core::Future<
@@ -1666,31 +1555,30 @@ impl tower_service::Service<Request<Body>> for Svc {
}
}
#[test]
fn http2_service_poll_ready_error_sends_goaway() {
#[tokio::test]
#[ignore] // sometimes ECONNRESET wins the race
async fn http2_service_poll_ready_error_sends_goaway() {
use std::error::Error;
let _ = pretty_env_logger::try_init();
let server = hyper::Server::bind(&([127, 0, 0, 1], 0).into())
.http2_only(true)
.serve(make_service_fn(|_| async move { Ok::<_, BoxError>(Svc) }));
.serve(make_service_fn(|_| async move { Ok::<_, BoxError>(Http2ReadyErrorSvc) }));
let addr_str = format!("http://{}", server.local_addr());
let mut rt = Runtime::new().expect("runtime new");
tokio::task::spawn(async move {
server.await.expect("server");
});
rt.spawn(server
.map_err(|e| unreachable!("server shouldn't error: {:?}", e))
.map(|_| ()));
let err = rt.block_on({
let client = Client::builder()
.http2_only(true)
.build_http::<hyper::Body>();
let uri = addr_str.parse().expect("server addr should parse");
client.get(uri)
}).unwrap_err();
let err = dbg!(Client::builder()
.http2_only(true)
.build_http::<hyper::Body>()
.get(uri)
.await
.expect_err("client.get should fail"));
// client request should have gotten the specific GOAWAY error...
let h2_err = err
@@ -1843,11 +1731,14 @@ impl<'a> Drop for ReplyBuilder<'a> {
impl Drop for Serve {
fn drop(&mut self) {
drop(self.shutdown_signal.take());
drop(self.thread.take());
/*
let r = self.thread.take().unwrap().join();
if let Err(ref e) = r {
println!("{:?}", e);
}
r.unwrap();
*/
}
}
@@ -1877,7 +1768,7 @@ enum Msg {
impl tower_service::Service<Request<Body>> for TestService {
type Response = Response<Body>;
type Error = BoxError;
type Future = BoxFuture<'static, Result<Response<Body>, BoxError>>;
type Future = Pin<Box<dyn Future<Output = Result<Response<Body>, BoxError>> + Send>>;
fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
Ok(()).into()
@@ -1886,7 +1777,8 @@ impl tower_service::Service<Request<Body>> for TestService {
fn call(&mut self, mut req: Request<Body>) -> Self::Future {
let tx = self.tx.clone();
let replies = self.reply.clone();
hyper::rt::spawn(async move {
Box::pin(async move {
while let Some(chunk) = req.body_mut().next().await {
match chunk {
Ok(chunk) => {
@@ -1894,15 +1786,13 @@ impl tower_service::Service<Request<Body>> for TestService {
},
Err(err) => {
tx.send(Msg::Error(err)).unwrap();
return;
return Err("req body error".into());
},
}
}
tx.send(Msg::End).unwrap();
});
Box::pin(async move {
TestService::build_reply(replies)
})
}
@@ -1940,7 +1830,7 @@ struct HelloWorld;
impl tower_service::Service<Request<Body>> for HelloWorld {
type Response = Response<Body>;
type Error = hyper::Error;
type Future = BoxFuture<'static, Result<Response<Body>, Self::Error>>;
type Future = future::Ready<Result<Response<Body>, Self::Error>>;
fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
Ok(()).into()
@@ -1948,7 +1838,7 @@ impl tower_service::Service<Request<Body>> for HelloWorld {
fn call(&mut self, _req: Request<Body>) -> Self::Future {
let response = Response::new(HELLO.into());
future::ok(response).boxed()
future::ok(response)
}
}
@@ -2021,6 +1911,14 @@ impl ServeOptions {
let thread = thread::Builder::new()
.name(thread_name)
.spawn(move || {
let mut rt = tokio::runtime::Builder::new()
.enable_io()
.enable_time()
.basic_scheduler()
.build()
.expect("rt new");
rt.block_on(async move {
let service = make_service_fn(|_| {
let msg_tx = msg_tx.clone();
let reply_rx = reply_rx.clone();
@@ -2040,15 +1938,12 @@ impl ServeOptions {
server.local_addr()
).expect("server addr tx");
let fut = server
server
.with_graceful_shutdown(async {
shutdown_rx.await.ok();
});
let mut rt = Runtime::new().expect("rt new");
rt
.block_on(fut)
.unwrap();
let _ = shutdown_rx.await;
})
.await
}).expect("serve()");
})
.expect("thread spawn");
@@ -2076,7 +1971,7 @@ fn has_header(msg: &str, name: &str) -> bool {
fn tcp_bind(addr: &SocketAddr) -> ::tokio::io::Result<TcpListener> {
let std_listener = StdTcpListener::bind(addr).unwrap();
TcpListener::from_std(std_listener, &Handle::default())
TcpListener::from_std(std_listener)
}
fn read_until<R, F>(io: &mut R, func: F) -> io::Result<Vec<u8>>

View File

@@ -9,9 +9,7 @@ use hyper::service::{make_service_fn, service_fn};
pub use std::net::SocketAddr;
pub use futures_util::{future, FutureExt as _, StreamExt as _, TryFutureExt as _, TryStreamExt as _};
//pub use self::futures_channel::oneshot;
pub use hyper::{HeaderMap, StatusCode};
pub use tokio::runtime::current_thread::Runtime;
macro_rules! t {
(
@@ -303,8 +301,16 @@ pub struct __TestConfig {
pub fn __run_test(cfg: __TestConfig) {
let _ = pretty_env_logger::try_init();
let mut rt = Runtime::new().expect("new rt");
tokio::runtime::Builder::new()
.enable_io()
.enable_time()
.basic_scheduler()
.build()
.expect("new rt")
.block_on(async_test(cfg));
}
async fn async_test(cfg: __TestConfig) {
assert_eq!(cfg.client_version, cfg.server_version);
let version = if cfg.client_version == 2 {
@@ -348,8 +354,7 @@ pub fn __run_test(cfg: __TestConfig) {
func(&req.headers());
}
let sbody = sreq.body;
req.into_body()
.try_concat()
concat(req.into_body())
.map_ok(move |body| {
assert_eq!(body.as_ref(), sbody.as_slice(), "client body");
@@ -369,7 +374,7 @@ pub fn __run_test(cfg: __TestConfig) {
let mut addr = server.local_addr();
rt.spawn(server.map(|result| {
tokio::task::spawn(server.map(|result| {
let _ = result.expect("server error");
}));
@@ -379,7 +384,7 @@ pub fn __run_test(cfg: __TestConfig) {
dst: addr,
version: cfg.server_version,
});
rt.spawn(proxy);
tokio::task::spawn(proxy);
addr = proxy_addr;
}
@@ -403,7 +408,7 @@ pub fn __run_test(cfg: __TestConfig) {
for func in &cheaders {
func(&res.headers());
}
res.into_body().try_concat()
concat(res.into_body())
})
.map_ok(move |body| {
assert_eq!(body.as_ref(), cbody.as_slice(), "server body");
@@ -435,7 +440,7 @@ pub fn __run_test(cfg: __TestConfig) {
Box::pin(client_futures.map(|_| ()))
};
rt.block_on(client_futures);
client_futures.await;
}
struct ProxyConfig {
@@ -470,3 +475,11 @@ fn naive_proxy(cfg: ProxyConfig) -> (SocketAddr, impl Future<Output = ()>) {
let proxy_addr = srv.local_addr();
(proxy_addr, srv.map(|res| res.expect("proxy error")))
}
async fn concat(mut body: Body) -> Result<hyper::Chunk, hyper::Error> {
let mut vec = Vec::new();
while let Some(chunk) = body.next().await {
vec.extend_from_slice(&chunk?);
}
Ok(vec.into())
}