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

@@ -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| {
assert!(Pin::new(&mut closes).poll_next(ctx).is_pending());
Poll::Ready(Ok::<_, ()>(()))
})).unwrap();
}
future::poll_fn(|ctx| {
assert!(Pin::new(&mut closes).poll_next(ctx).is_pending());
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 {