feat(lib): update Tokio, bytes, http, h2, and http-body
This commit is contained in:
		| @@ -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 | ||||
|   | ||||
| @@ -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; | ||||
|   | ||||
| @@ -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 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, 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 = 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 } | ||||
|                     )), | ||||
|   | ||||
| @@ -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}; | ||||
|   | ||||
| @@ -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,56 +283,43 @@ 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") | ||||
|                 .expect_err("promise should error"); | ||||
|             match (err.0.kind(), err.1) { | ||||
|                 (&crate::error::Kind::Canceled, Some(_)) => (), | ||||
|                 e => panic!("expected Error::Cancel(_), found {:?}", e), | ||||
|             } | ||||
|         }); | ||||
|         let fulfilled = promise.await; | ||||
|         let err = fulfilled | ||||
|             .expect("fulfilled") | ||||
|             .expect_err("promise should error"); | ||||
|         match (err.0.kind(), err.1) { | ||||
|             (&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 { | ||||
|   | ||||
| @@ -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 | ||||
|   | ||||
| @@ -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"), | ||||
|             }; | ||||
|         }) | ||||
|         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 { | ||||
|             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); | ||||
|         }); | ||||
|         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()); | ||||
|         }); | ||||
|         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)); | ||||
|         }); | ||||
|         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)); | ||||
|         }); | ||||
|         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); | ||||
|         }); | ||||
|         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); | ||||
|   | ||||
| @@ -63,7 +63,7 @@ where | ||||
|                                 if let Err(e) = conn.await { | ||||
|                                     debug!("connection error: {:?}", e); | ||||
|                                 } | ||||
|                             })?; | ||||
|                             }); | ||||
|                             Ok(sr) | ||||
|                         }, | ||||
|                         Err(e) => Err(e) | ||||
|   | ||||
		Reference in New Issue
	
	Block a user