feat(lib): redesign API to use Futures and Tokio
There are many changes involved with this, but let's just talk about
user-facing changes.
- Creating a `Client` and `Server` now needs a Tokio `Core` event loop
to attach to.
- `Request` and `Response` both no longer implement the
`std::io::{Read,Write}` traits, but instead represent their bodies as a
`futures::Stream` of items, where each item is a `Chunk`.
- The `Client.request` method now takes a `Request`, instead of being
used as a builder, and returns a `Future` that resolves to `Response`.
- The `Handler` trait for servers is no more, and instead the Tokio
`Service` trait is used. This allows interoperability with generic
middleware.
BREAKING CHANGE: A big sweeping set of breaking changes.
			
			
This commit is contained in:
		| @@ -1,59 +1,59 @@ | ||||
| use std::collections::hash_map::{HashMap, Entry}; | ||||
| use std::hash::Hash; | ||||
| use std::fmt; | ||||
| use std::io; | ||||
| use std::net::SocketAddr; | ||||
| //use std::net::SocketAddr; | ||||
|  | ||||
| use rotor::mio::tcp::TcpStream; | ||||
| use futures::{Future, Poll, Async}; | ||||
| use tokio::io::Io; | ||||
| use tokio::reactor::Handle; | ||||
| use tokio::net::{TcpStream, TcpStreamNew}; | ||||
| use tokio_service::Service; | ||||
| use url::Url; | ||||
|  | ||||
| use net::{HttpStream, HttpsStream, Transport, SslClient}; | ||||
| use super::dns::Dns; | ||||
| use super::Registration; | ||||
| use super::dns; | ||||
|  | ||||
| /// A connector creates a Transport to a remote address.. | ||||
| pub trait Connect { | ||||
|     /// Type of Transport to create | ||||
|     type Output: Transport; | ||||
|     /// The key used to determine if an existing socket can be used. | ||||
|     type Key: Eq + Hash + Clone + fmt::Debug; | ||||
|     /// Returns the key based off the Url. | ||||
|     fn key(&self, &Url) -> Option<Self::Key>; | ||||
| /// A connector creates an Io to a remote address.. | ||||
| /// | ||||
| /// This trait is not implemented directly, and only exists to make | ||||
| /// the intent clearer. A connector should implement `Service` with | ||||
| /// `Request=Url` and `Response: Io` instead. | ||||
| pub trait Connect: Service<Request=Url, Error=io::Error> + 'static { | ||||
|     /// The connected Io Stream. | ||||
|     type Output: Io + 'static; | ||||
|     /// A Future that will resolve to the connected Stream. | ||||
|     type Future: Future<Item=Self::Output, Error=io::Error> + 'static; | ||||
|     /// Connect to a remote address. | ||||
|     fn connect(&mut self, &Url) -> io::Result<Self::Key>; | ||||
|     /// Returns a connected socket and associated host. | ||||
|     fn connected(&mut self) -> Option<(Self::Key, io::Result<Self::Output>)>; | ||||
|     #[doc(hidden)] | ||||
|     /// Configure number of dns workers to use. | ||||
|     fn dns_workers(&mut self, usize); | ||||
|     #[doc(hidden)] | ||||
|     fn register(&mut self, Registration); | ||||
|     fn connect(&self, Url) -> <Self as Connect>::Future; | ||||
| } | ||||
|  | ||||
| /// A connector for the `http` scheme. | ||||
| pub struct HttpConnector { | ||||
|     dns: Option<Dns>, | ||||
|     threads: usize, | ||||
|     resolving: HashMap<String, Vec<(&'static str, String, u16)>>, | ||||
| } | ||||
| impl<T> Connect for T | ||||
| where T: Service<Request=Url, Error=io::Error> + 'static, | ||||
|       T::Response: Io, | ||||
|       T::Future: Future<Error=io::Error>, | ||||
| { | ||||
|     type Output = T::Response; | ||||
|     type Future = T::Future; | ||||
|  | ||||
| impl HttpConnector { | ||||
|     /// Set the number of resolver threads. | ||||
|     /// | ||||
|     /// Default is 4. | ||||
|     pub fn threads(mut self, threads: usize) -> HttpConnector { | ||||
|         debug_assert!(self.dns.is_none(), "setting threads after Dns is created does nothing"); | ||||
|         self.threads = threads; | ||||
|         self | ||||
|     fn connect(&self, url: Url) -> <Self as Connect>::Future { | ||||
|         self.call(url) | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Default for HttpConnector { | ||||
|     fn default() -> HttpConnector { | ||||
| /// A connector for the `http` scheme. | ||||
| #[derive(Clone)] | ||||
| pub struct HttpConnector { | ||||
|     dns: dns::Dns, | ||||
|     handle: Handle, | ||||
| } | ||||
|  | ||||
| impl HttpConnector { | ||||
|  | ||||
|     /// Construct a new HttpConnector. | ||||
|     /// | ||||
|     /// Takes number of DNS worker threads. | ||||
|     pub fn new(threads: usize, handle: &Handle) -> HttpConnector { | ||||
|         HttpConnector { | ||||
|             dns: None, | ||||
|             threads: 4, | ||||
|             resolving: HashMap::new(), | ||||
|             dns: dns::Dns::new(threads), | ||||
|             handle: handle.clone(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -61,79 +61,115 @@ impl Default for HttpConnector { | ||||
| impl fmt::Debug for HttpConnector { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.debug_struct("HttpConnector") | ||||
|             .field("threads", &self.threads) | ||||
|             .field("resolving", &self.resolving) | ||||
|             .finish() | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Connect for HttpConnector { | ||||
|     type Output = HttpStream; | ||||
|     type Key = (&'static str, String, u16); | ||||
| impl Service for HttpConnector { | ||||
|     type Request = Url; | ||||
|     type Response = TcpStream; | ||||
|     type Error = io::Error; | ||||
|     type Future = HttpConnecting; | ||||
|  | ||||
|     fn dns_workers(&mut self, count: usize) { | ||||
|         self.threads = count; | ||||
|     } | ||||
|  | ||||
|     fn key(&self, url: &Url) -> Option<Self::Key> { | ||||
|         if url.scheme() == "http" { | ||||
|             Some(( | ||||
|                 "http", | ||||
|                 url.host_str().expect("http scheme must have host").to_owned(), | ||||
|                 url.port().unwrap_or(80), | ||||
|             )) | ||||
|         } else { | ||||
|             None | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn connect(&mut self, url: &Url) -> io::Result<Self::Key> { | ||||
|     fn call(&self, url: Url) -> Self::Future { | ||||
|         debug!("Http::connect({:?})", url); | ||||
|         if let Some(key) = self.key(url) { | ||||
|             let host = url.host_str().expect("http scheme must have a host"); | ||||
|             self.dns.as_ref().expect("dns workers lost").resolve(host); | ||||
|             self.resolving.entry(host.to_owned()).or_insert_with(Vec::new).push(key.clone()); | ||||
|             Ok(key) | ||||
|         } else { | ||||
|             Err(io::Error::new(io::ErrorKind::InvalidInput, "scheme must be http")) | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn connected(&mut self) -> Option<(Self::Key, io::Result<HttpStream>)> { | ||||
|         let (host, addrs) = match self.dns.as_ref().expect("dns workers lost").resolved() { | ||||
|             Ok(res) => res, | ||||
|             Err(_) => return None | ||||
|         let host = match url.host_str() { | ||||
|             Some(s) => s, | ||||
|             None => return HttpConnecting { | ||||
|                 state: State::Error(Some(io::Error::new(io::ErrorKind::InvalidInput, "invalid url"))), | ||||
|                 handle: self.handle.clone(), | ||||
|             }, | ||||
|         }; | ||||
|         //TODO: try all addrs | ||||
|         let addr = addrs.and_then(|mut addrs| Ok(addrs.next().unwrap())); | ||||
|         debug!("Http::resolved <- ({:?}, {:?})", host, addr); | ||||
|         if let Entry::Occupied(mut entry) = self.resolving.entry(host) { | ||||
|             let resolved = entry.get_mut().remove(0); | ||||
|             if entry.get().is_empty() { | ||||
|                 entry.remove(); | ||||
|             } | ||||
|             let port = resolved.2; | ||||
|             Some((resolved, addr.and_then(|addr| TcpStream::connect(&SocketAddr::new(addr, port)) | ||||
|                                                             .map(HttpStream)) | ||||
|                 )) | ||||
|         } else { | ||||
|             trace!("^--  resolved but not in hashmap?"); | ||||
|             None | ||||
|         let port = url.port_or_known_default().unwrap_or(80); | ||||
|  | ||||
|         HttpConnecting { | ||||
|             state: State::Resolving(self.dns.resolve(host.into(), port)), | ||||
|             handle: self.handle.clone(), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn register(&mut self, reg: Registration) { | ||||
|         self.dns = Some(Dns::new(reg.notify, self.threads)); | ||||
| } | ||||
|  | ||||
| /// A Future representing work to connect to a URL. | ||||
| pub struct HttpConnecting { | ||||
|     state: State, | ||||
|     handle: Handle, | ||||
| } | ||||
|  | ||||
| enum State { | ||||
|     Resolving(dns::Query), | ||||
|     Connecting(ConnectingTcp), | ||||
|     Error(Option<io::Error>), | ||||
| } | ||||
|  | ||||
| impl Future for HttpConnecting { | ||||
|     type Item = TcpStream; | ||||
|     type Error = io::Error; | ||||
|  | ||||
|     fn poll(&mut self) -> Poll<Self::Item, Self::Error> { | ||||
|         loop { | ||||
|             let state; | ||||
|             match self.state { | ||||
|                 State::Resolving(ref mut query) => { | ||||
|                     match try!(query.poll()) { | ||||
|                         Async::NotReady => return Ok(Async::NotReady), | ||||
|                         Async::Ready(addrs) => { | ||||
|                             state = State::Connecting(ConnectingTcp { | ||||
|                                 addrs: addrs, | ||||
|                                 current: None, | ||||
|                             }) | ||||
|                         } | ||||
|                     }; | ||||
|                 }, | ||||
|                 State::Connecting(ref mut c) => return c.poll(&self.handle).map_err(From::from), | ||||
|                 State::Error(ref mut e) => return Err(e.take().expect("polled more than once")), | ||||
|             } | ||||
|             self.state = state; | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| /// A connector that can protect HTTP streams using SSL. | ||||
| #[derive(Debug, Default)] | ||||
| pub struct HttpsConnector<S: SslClient> { | ||||
|     http: HttpConnector, | ||||
|     ssl: S | ||||
| impl fmt::Debug for HttpConnecting { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.pad("HttpConnecting") | ||||
|     } | ||||
| } | ||||
|  | ||||
| struct ConnectingTcp { | ||||
|     addrs: dns::IpAddrs, | ||||
|     current: Option<TcpStreamNew>, | ||||
| } | ||||
|  | ||||
| impl ConnectingTcp { | ||||
|     // not a Future, since passing a &Handle to poll | ||||
|     fn poll(&mut self, handle: &Handle) -> Poll<TcpStream, io::Error> { | ||||
|         let mut err = None; | ||||
|         loop { | ||||
|             if let Some(ref mut current) = self.current { | ||||
|                 match current.poll() { | ||||
|                     Ok(ok) => return Ok(ok), | ||||
|                     Err(e) => { | ||||
|                         trace!("connect error {:?}", e); | ||||
|                         err = Some(e); | ||||
|                         if let Some(addr) = self.addrs.next() { | ||||
|                             debug!("connecting to {:?}", addr); | ||||
|                             *current = TcpStream::connect(&addr, handle); | ||||
|                             continue; | ||||
|                         } | ||||
|                     } | ||||
|                 } | ||||
|             } else if let Some(addr) = self.addrs.next() { | ||||
|                 debug!("connecting to {:?}", addr); | ||||
|                 self.current = Some(TcpStream::connect(&addr, handle)); | ||||
|                 continue; | ||||
|             } | ||||
|  | ||||
|             return Err(err.take().expect("missing connect error")); | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| /* | ||||
| impl<S: SslClient> HttpsConnector<S> { | ||||
|     /// Create a new connector using the provided SSL implementation. | ||||
|     pub fn new(s: S) -> HttpsConnector<S> { | ||||
| @@ -143,80 +179,22 @@ impl<S: SslClient> HttpsConnector<S> { | ||||
|         } | ||||
|     } | ||||
| } | ||||
| */ | ||||
|  | ||||
| impl<S: SslClient> Connect for HttpsConnector<S> { | ||||
|     type Output = HttpsStream<S::Stream>; | ||||
|     type Key = (&'static str, String, u16); | ||||
| #[cfg(test)] | ||||
| mod tests { | ||||
|     use std::io; | ||||
|     use tokio::reactor::Core; | ||||
|     use url::Url; | ||||
|     use super::{Connect, HttpConnector}; | ||||
|  | ||||
|     fn dns_workers(&mut self, count: usize) { | ||||
|         self.http.dns_workers(count) | ||||
|     #[test] | ||||
|     fn test_non_http_url() { | ||||
|         let mut core = Core::new().unwrap(); | ||||
|         let url = Url::parse("file:///home/sean/foo.txt").unwrap(); | ||||
|         let connector = HttpConnector::new(1, &core.handle()); | ||||
|  | ||||
|         assert_eq!(core.run(connector.connect(url)).unwrap_err().kind(), io::ErrorKind::InvalidInput); | ||||
|     } | ||||
|  | ||||
|     fn key(&self, url: &Url) -> Option<Self::Key> { | ||||
|         let scheme = match url.scheme() { | ||||
|             "http" => "http", | ||||
|             "https" => "https", | ||||
|             _ => return None | ||||
|         }; | ||||
|         Some(( | ||||
|             scheme, | ||||
|             url.host_str().expect("http scheme must have host").to_owned(), | ||||
|             url.port_or_known_default().expect("http scheme must have a port"), | ||||
|         )) | ||||
|     } | ||||
|  | ||||
|     fn connect(&mut self, url: &Url) -> io::Result<Self::Key> { | ||||
|         debug!("Https::connect({:?})", url); | ||||
|         if let Some(key) = self.key(url) { | ||||
|             let host = url.host_str().expect("http scheme must have a host"); | ||||
|             self.http.dns.as_ref().expect("dns workers lost").resolve(host); | ||||
|             self.http.resolving.entry(host.to_owned()).or_insert_with(Vec::new).push(key.clone()); | ||||
|             Ok(key) | ||||
|         } else { | ||||
|             Err(io::Error::new(io::ErrorKind::InvalidInput, "scheme must be http or https")) | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn connected(&mut self) -> Option<(Self::Key, io::Result<Self::Output>)> { | ||||
|         self.http.connected().map(|(key, res)| { | ||||
|             let res = res.and_then(|http| { | ||||
|                 if key.0 == "https" { | ||||
|                     self.ssl.wrap_client(http, &key.1) | ||||
|                         .map(HttpsStream::Https) | ||||
|                         .map_err(|e| match e { | ||||
|                             ::Error::Io(e) => e, | ||||
|                             e => io::Error::new(io::ErrorKind::Other, e) | ||||
|                         }) | ||||
|                 } else { | ||||
|                     Ok(HttpsStream::Http(http)) | ||||
|                 } | ||||
|             }); | ||||
|             (key, res) | ||||
|         }) | ||||
|     } | ||||
|  | ||||
|     fn register(&mut self, reg: Registration) { | ||||
|         self.http.register(reg); | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[cfg(not(any(feature = "openssl", feature = "security-framework")))] | ||||
| #[doc(hidden)] | ||||
| pub type DefaultConnector = HttpConnector; | ||||
|  | ||||
| #[cfg(all(feature = "openssl", not(feature = "security-framework")))] | ||||
| #[doc(hidden)] | ||||
| pub type DefaultConnector = HttpsConnector<::net::Openssl>; | ||||
|  | ||||
| #[cfg(feature = "security-framework")] | ||||
| #[doc(hidden)] | ||||
| pub type DefaultConnector = HttpsConnector<::net::SecureTransportClient>; | ||||
|  | ||||
| #[doc(hidden)] | ||||
| pub type DefaultTransport = <DefaultConnector as Connect>::Output; | ||||
|  | ||||
| fn _assert_defaults() { | ||||
|     fn _assert<T, U>() where T: Connect<Output=U>, U: Transport {} | ||||
|  | ||||
|     _assert::<DefaultConnector, DefaultTransport>(); | ||||
| } | ||||
|   | ||||
| @@ -1,96 +1,53 @@ | ||||
| use std::io; | ||||
| use std::net::{IpAddr, SocketAddr, ToSocketAddrs}; | ||||
| use std::thread; | ||||
| use std::net::{SocketAddr, ToSocketAddrs}; | ||||
| use std::vec; | ||||
|  | ||||
| use ::spmc; | ||||
|  | ||||
| use http::channel; | ||||
| use ::futures::{Future, Poll}; | ||||
| use ::futures_cpupool::{CpuPool, CpuFuture}; | ||||
|  | ||||
| #[derive(Clone)] | ||||
| pub struct Dns { | ||||
|     tx: spmc::Sender<String>, | ||||
|     rx: channel::Receiver<Answer>, | ||||
|     pool: CpuPool, | ||||
| } | ||||
|  | ||||
| pub type Answer = (String, io::Result<IpAddrs>); | ||||
| impl Dns { | ||||
|     pub fn new(threads: usize) -> Dns { | ||||
|         Dns { | ||||
|             pool: CpuPool::new(threads) | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub fn resolve(&self, host: String, port: u16) -> Query { | ||||
|         Query(self.pool.spawn_fn(move || work(host, port))) | ||||
|     } | ||||
| } | ||||
|  | ||||
| pub struct Query(CpuFuture<IpAddrs, io::Error>); | ||||
|  | ||||
| impl Future for Query { | ||||
|     type Item = IpAddrs; | ||||
|     type Error = io::Error; | ||||
|  | ||||
|     fn poll(&mut self) -> Poll<Self::Item, Self::Error> { | ||||
|         self.0.poll() | ||||
|     } | ||||
| } | ||||
|  | ||||
| pub struct IpAddrs { | ||||
|     iter: vec::IntoIter<SocketAddr>, | ||||
| } | ||||
|  | ||||
| impl Iterator for IpAddrs { | ||||
|     type Item = IpAddr; | ||||
|     type Item = SocketAddr; | ||||
|     #[inline] | ||||
|     fn next(&mut self) -> Option<IpAddr> { | ||||
|         self.iter.next().map(|addr| addr.ip()) | ||||
|     fn next(&mut self) -> Option<SocketAddr> { | ||||
|         self.iter.next() | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Dns { | ||||
|     pub fn new(notify: (channel::Sender<Answer>, channel::Receiver<Answer>), threads: usize) -> Dns { | ||||
|         let (tx, rx) = spmc::channel(); | ||||
|         for _ in 0..threads { | ||||
|             work(rx.clone(), notify.0.clone()); | ||||
|         } | ||||
|         Dns { | ||||
|             tx: tx, | ||||
|             rx: notify.1, | ||||
|         } | ||||
|     } | ||||
| pub type Answer = io::Result<IpAddrs>; | ||||
|  | ||||
|     pub fn resolve<T: Into<String>>(&self, hostname: T) { | ||||
|         self.tx.send(hostname.into()).expect("DNS workers all died unexpectedly"); | ||||
|     } | ||||
|  | ||||
|     pub fn resolved(&self) -> Result<Answer, channel::TryRecvError> { | ||||
|         self.rx.try_recv() | ||||
|     } | ||||
| } | ||||
|  | ||||
| fn work(rx: spmc::Receiver<String>, notify: channel::Sender<Answer>) { | ||||
|     thread::Builder::new().name(String::from("hyper-dns")).spawn(move || { | ||||
|         let mut worker = Worker::new(rx, notify); | ||||
|         let rx = worker.rx.as_ref().expect("Worker lost rx"); | ||||
|         let notify = worker.notify.as_ref().expect("Worker lost notify"); | ||||
|         while let Ok(host) = rx.recv() { | ||||
|             debug!("resolve {:?}", host); | ||||
|             let res = match (&*host, 80).to_socket_addrs().map(|i| IpAddrs{ iter: i }) { | ||||
|                 Ok(addrs) => (host, Ok(addrs)), | ||||
|                 Err(e) => (host, Err(e)) | ||||
|             }; | ||||
|  | ||||
|             if let Err(_) = notify.send(res) { | ||||
|                 break; | ||||
|             } | ||||
|         } | ||||
|         worker.shutdown = true; | ||||
|     }).expect("spawn dns thread"); | ||||
| } | ||||
|  | ||||
| struct Worker { | ||||
|     rx: Option<spmc::Receiver<String>>, | ||||
|     notify: Option<channel::Sender<Answer>>, | ||||
|     shutdown: bool, | ||||
| } | ||||
|  | ||||
| impl Worker { | ||||
|     fn new(rx: spmc::Receiver<String>, notify: channel::Sender<Answer>) -> Worker { | ||||
|         Worker { | ||||
|             rx: Some(rx), | ||||
|             notify: Some(notify), | ||||
|             shutdown: false, | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Drop for Worker { | ||||
|     fn drop(&mut self) { | ||||
|         if !self.shutdown { | ||||
|             trace!("Worker.drop panicked, restarting"); | ||||
|             work(self.rx.take().expect("Worker lost rx"), | ||||
|                 self.notify.take().expect("Worker lost notify")); | ||||
|         } else { | ||||
|             trace!("Worker.drop shutdown, closing"); | ||||
|         } | ||||
|     } | ||||
| fn work(hostname: String, port: u16) -> Answer { | ||||
|     debug!("resolve {:?}:{:?}", hostname, port); | ||||
|     (&*hostname, port).to_socket_addrs().map(|i| IpAddrs { iter: i }) | ||||
| } | ||||
|   | ||||
| @@ -3,51 +3,49 @@ | ||||
| //! The HTTP `Client` uses asynchronous IO, and utilizes the `Handler` trait | ||||
| //! to convey when IO events are available for a given request. | ||||
|  | ||||
| use std::collections::{VecDeque, HashMap}; | ||||
| use std::cell::RefCell; | ||||
| use std::fmt; | ||||
| use std::io; | ||||
| use std::marker::PhantomData; | ||||
| use std::sync::mpsc; | ||||
| use std::thread; | ||||
| use std::rc::Rc; | ||||
| use std::time::Duration; | ||||
|  | ||||
| use rotor::{self, Scope, EventSet, PollOpt}; | ||||
| use futures::{Poll, Async, Future}; | ||||
| use relay; | ||||
| use tokio::io::Io; | ||||
| use tokio::reactor::Handle; | ||||
| use tokio_proto::BindClient; | ||||
| use tokio_proto::streaming::Message; | ||||
| use tokio_proto::streaming::pipeline::ClientProto; | ||||
| use tokio_proto::util::client_proxy::ClientProxy; | ||||
| pub use tokio_service::Service; | ||||
|  | ||||
| use header::Host; | ||||
| use http::{self, Next, RequestHead, ReadyResult}; | ||||
| use net::Transport; | ||||
| use header::{Headers, Host}; | ||||
| use http::{self, TokioBody}; | ||||
| use method::Method; | ||||
| use self::pool::{Pool, Pooled}; | ||||
| use uri::RequestUri; | ||||
| use {Url}; | ||||
|  | ||||
| pub use self::connect::{Connect, DefaultConnector, HttpConnector, HttpsConnector, DefaultTransport}; | ||||
| pub use self::connect::{HttpConnector, Connect}; | ||||
| pub use self::request::Request; | ||||
| pub use self::response::Response; | ||||
|  | ||||
| mod connect; | ||||
| mod dns; | ||||
| mod pool; | ||||
| mod request; | ||||
| mod response; | ||||
|  | ||||
| /// A Client to make outgoing HTTP requests. | ||||
| pub struct Client<H> { | ||||
|     tx: http::channel::Sender<Notify<H>>, | ||||
| // If the Connector is clone, then the Client can be clone easily. | ||||
| #[derive(Clone)] | ||||
| pub struct Client<C> { | ||||
|     connector: C, | ||||
|     handle: Handle, | ||||
|     pool: Pool<TokioClient>, | ||||
| } | ||||
|  | ||||
| impl<H> Clone for Client<H> { | ||||
|     fn clone(&self) -> Client<H> { | ||||
|         Client { | ||||
|             tx: self.tx.clone() | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<H> fmt::Debug for Client<H> { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.pad("Client") | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<H> Client<H> { | ||||
| impl Client<HttpConnector> { | ||||
|     /// Configure a Client. | ||||
|     /// | ||||
|     /// # Example | ||||
| @@ -56,116 +54,218 @@ impl<H> Client<H> { | ||||
|     /// # use hyper::Client; | ||||
|     /// let client = Client::configure() | ||||
|     ///     .keep_alive(true) | ||||
|     ///     .max_sockets(10_000) | ||||
|     ///     .build().unwrap(); | ||||
|     /// ``` | ||||
|     #[inline] | ||||
|     pub fn configure() -> Config<DefaultConnector> { | ||||
|     pub fn configure() -> Config<UseDefaultConnector> { | ||||
|         Config::default() | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<H: Handler<<DefaultConnector as Connect>::Output>> Client<H> { | ||||
| impl Client<HttpConnector> { | ||||
|     /// Create a new Client with the default config. | ||||
|     #[inline] | ||||
|     pub fn new() -> ::Result<Client<H>> { | ||||
|         Client::<H>::configure().build() | ||||
|     pub fn new(handle: &Handle) -> Client<HttpConnector> { | ||||
|         Client::configure().build(handle) | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<H: Send> Client<H> { | ||||
| impl<C: Connect> Client<C> { | ||||
|     /// Create a new client with a specific connector. | ||||
|     fn configured<T, C>(config: Config<C>) -> ::Result<Client<H>> | ||||
|     where H: Handler<T>, | ||||
|           T: Transport, | ||||
|           C: Connect<Output=T> + Send + 'static { | ||||
|         let mut rotor_config = rotor::Config::new(); | ||||
|         rotor_config.slab_capacity(config.max_sockets); | ||||
|         rotor_config.mio().notify_capacity(config.max_sockets); | ||||
|         let keep_alive = config.keep_alive; | ||||
|         let connect_timeout = config.connect_timeout; | ||||
|         let mut loop_ = try!(rotor::Loop::new(&rotor_config)); | ||||
|         let mut notifier = None; | ||||
|         let mut connector = config.connector; | ||||
|         connector.dns_workers(config.dns_workers); | ||||
|         { | ||||
|             let not = &mut notifier; | ||||
|             loop_.add_machine_with(move |scope| { | ||||
|                 let (tx, rx) = http::channel::new(scope.notifier()); | ||||
|                 let (dns_tx, dns_rx) = http::channel::share(&tx); | ||||
|                 *not = Some(tx); | ||||
|                 connector.register(Registration { | ||||
|                     notify: (dns_tx, dns_rx), | ||||
|                 }); | ||||
|                 rotor::Response::ok(ClientFsm::Connector(connector, rx)) | ||||
|             }).unwrap(); | ||||
|     #[inline] | ||||
|     fn configured(config: Config<C>, handle: &Handle) -> Client<C> { | ||||
|         Client { | ||||
|             connector: config.connector, | ||||
|             handle: handle.clone(), | ||||
|             pool: Pool::new(config.keep_alive, config.keep_alive_timeout), | ||||
|         } | ||||
|  | ||||
|         let notifier = notifier.expect("loop.add_machine_with failed"); | ||||
|         let _handle = try!(thread::Builder::new().name("hyper-client".to_owned()).spawn(move || { | ||||
|             loop_.run(Context { | ||||
|                 connect_timeout: connect_timeout, | ||||
|                 keep_alive: keep_alive, | ||||
|                 idle_conns: HashMap::new(), | ||||
|                 queue: HashMap::new(), | ||||
|                 awaiting_slot: VecDeque::new(), | ||||
|             }).unwrap() | ||||
|         })); | ||||
|  | ||||
|         Ok(Client { | ||||
|             //handle: Some(handle), | ||||
|             tx: notifier, | ||||
|         }) | ||||
|     } | ||||
|  | ||||
|     /// Build a new request using this Client. | ||||
|     /// | ||||
|     /// ## Error | ||||
|     /// | ||||
|     /// If the event loop thread has died, or the queue is full, a `ClientError` | ||||
|     /// will be returned. | ||||
|     pub fn request(&self, url: Url, handler: H) -> Result<(), ClientError<H>> { | ||||
|         self.tx.send(Notify::Connect(url, handler)).map_err(|e| { | ||||
|             match e.0 { | ||||
|                 Some(Notify::Connect(url, handler)) => ClientError(Some((url, handler))), | ||||
|                 _ => ClientError(None) | ||||
|     /// Send a GET Request using this Client. | ||||
|     #[inline] | ||||
|     pub fn get(&self, url: Url) -> FutureResponse { | ||||
|         self.request(Request::new(Method::Get, url)) | ||||
|     } | ||||
|  | ||||
|     /// Send a constructed Request using this Client. | ||||
|     #[inline] | ||||
|     pub fn request(&self, req: Request) -> FutureResponse { | ||||
|         self.call(req) | ||||
|     } | ||||
| } | ||||
|  | ||||
| /// A `Future` that will resolve to an HTTP Response. | ||||
| pub struct FutureResponse(Box<Future<Item=Response, Error=::Error> + 'static>); | ||||
|  | ||||
| impl fmt::Debug for FutureResponse { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.pad("Future<Response>") | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Future for FutureResponse { | ||||
|     type Item = Response; | ||||
|     type Error = ::Error; | ||||
|  | ||||
|     fn poll(&mut self) -> Poll<Self::Item, Self::Error> { | ||||
|         self.0.poll() | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<C: Connect> Service for Client<C> { | ||||
|     type Request = Request; | ||||
|     type Response = Response; | ||||
|     type Error = ::Error; | ||||
|     type Future = FutureResponse; | ||||
|  | ||||
|     fn call(&self, req: Request) -> Self::Future { | ||||
|         let url = req.url().clone(); | ||||
|  | ||||
|         let (mut head, body) = request::split(req); | ||||
|         let mut headers = Headers::new(); | ||||
|         headers.set(Host { | ||||
|             hostname: url.host_str().unwrap().to_owned(), | ||||
|             port: url.port().or(None), | ||||
|         }); | ||||
|         headers.extend(head.headers.iter()); | ||||
|         head.subject.1 = RequestUri::AbsolutePath { | ||||
|             path: url.path().to_owned(), | ||||
|             query: url.query().map(ToOwned::to_owned), | ||||
|         }; | ||||
|         head.headers = headers; | ||||
|  | ||||
|         let checkout = self.pool.checkout(&url[..::url::Position::BeforePath]); | ||||
|         let connect = { | ||||
|             let handle = self.handle.clone(); | ||||
|             let pool = self.pool.clone(); | ||||
|             let pool_key = Rc::new(url[..::url::Position::BeforePath].to_owned()); | ||||
|             self.connector.connect(url) | ||||
|                 .map(move |io| { | ||||
|                     let (tx, rx) = relay::channel(); | ||||
|                     let client = HttpClient { | ||||
|                         client_rx: RefCell::new(Some(rx)), | ||||
|                     }.bind_client(&handle, io); | ||||
|                     let pooled = pool.pooled(pool_key, client); | ||||
|                     tx.complete(pooled.clone()); | ||||
|                     pooled | ||||
|                 }) | ||||
|         }; | ||||
|  | ||||
|         let race = checkout.select(connect) | ||||
|             .map(|(client, _work)| client) | ||||
|             .map_err(|(e, _work)| { | ||||
|                 // the Pool Checkout cannot error, so the only error | ||||
|                 // is from the Connector | ||||
|                 // XXX: should wait on the Checkout? Problem is | ||||
|                 // that if the connector is failing, it may be that we | ||||
|                 // never had a pooled stream at all | ||||
|                 e.into() | ||||
|             }); | ||||
|         let req = race.and_then(move |client| { | ||||
|             let msg = match body { | ||||
|                 Some(body) => { | ||||
|                     Message::WithBody(head, body.into()) | ||||
|                 }, | ||||
|                 None => Message::WithoutBody(head), | ||||
|             }; | ||||
|             client.call(msg) | ||||
|         }); | ||||
|         FutureResponse(Box::new(req.map(|msg| { | ||||
|             match msg { | ||||
|                 Message::WithoutBody(head) => response::new(head, None), | ||||
|                 Message::WithBody(head, body) => response::new(head, Some(body.into())), | ||||
|             } | ||||
|         }) | ||||
|         }))) | ||||
|     } | ||||
|  | ||||
|     /// Close the Client loop. | ||||
|     pub fn close(self) { | ||||
|         // Most errors mean that the Receivers are already dead, which would | ||||
|         // imply the EventLoop panicked. | ||||
|         let _ = self.tx.send(Notify::Shutdown); | ||||
| } | ||||
|  | ||||
| impl<C> fmt::Debug for Client<C> { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.pad("Client") | ||||
|     } | ||||
| } | ||||
|  | ||||
| type TokioClient = ClientProxy<Message<http::RequestHead, TokioBody>, Message<http::ResponseHead, TokioBody>, ::Error>; | ||||
|  | ||||
| struct HttpClient { | ||||
|     client_rx: RefCell<Option<relay::Receiver<Pooled<TokioClient>>>>, | ||||
| } | ||||
|  | ||||
| impl<T: Io + 'static> ClientProto<T> for HttpClient { | ||||
|     type Request = http::RequestHead; | ||||
|     type RequestBody = http::Chunk; | ||||
|     type Response = http::ResponseHead; | ||||
|     type ResponseBody = http::Chunk; | ||||
|     type Error = ::Error; | ||||
|     type Transport = http::Conn<T, http::ClientTransaction, Pooled<TokioClient>>; | ||||
|     type BindTransport = BindingClient<T>; | ||||
|  | ||||
|     fn bind_transport(&self, io: T) -> Self::BindTransport { | ||||
|         BindingClient { | ||||
|             rx: self.client_rx.borrow_mut().take().expect("client_rx was lost"), | ||||
|             io: Some(io), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| struct BindingClient<T> { | ||||
|     rx: relay::Receiver<Pooled<TokioClient>>, | ||||
|     io: Option<T>, | ||||
| } | ||||
|  | ||||
| impl<T: Io + 'static> Future for BindingClient<T> { | ||||
|     type Item = http::Conn<T, http::ClientTransaction, Pooled<TokioClient>>; | ||||
|     type Error = io::Error; | ||||
|  | ||||
|     fn poll(&mut self) -> Poll<Self::Item, Self::Error> { | ||||
|         match self.rx.poll() { | ||||
|             Ok(Async::Ready(client)) => Ok(Async::Ready( | ||||
|                     http::Conn::new(self.io.take().expect("binding client io lost"), client) | ||||
|             )), | ||||
|             Ok(Async::NotReady) => Ok(Async::NotReady), | ||||
|             Err(_canceled) => unreachable!(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| /// Configuration for a Client | ||||
| #[derive(Debug, Clone)] | ||||
| pub struct Config<C> { | ||||
|     connect_timeout: Duration, | ||||
|     //connect_timeout: Duration, | ||||
|     connector: C, | ||||
|     keep_alive: bool, | ||||
|     keep_alive_timeout: Option<Duration>, | ||||
|     //TODO: make use of max_idle config | ||||
|     max_idle: usize, | ||||
|     max_sockets: usize, | ||||
|     dns_workers: usize, | ||||
| } | ||||
|  | ||||
| impl<C> Config<C> where C: Connect + Send + 'static { | ||||
| /// Phantom type used to signal that `Config` should create a `HttpConnector`. | ||||
| #[derive(Debug, Clone, Copy)] | ||||
| pub struct UseDefaultConnector(()); | ||||
|  | ||||
| impl Config<UseDefaultConnector> { | ||||
|     fn default() -> Config<UseDefaultConnector> { | ||||
|         Config { | ||||
|             //connect_timeout: Duration::from_secs(10), | ||||
|             connector: UseDefaultConnector(()), | ||||
|             keep_alive: true, | ||||
|             keep_alive_timeout: Some(Duration::from_secs(90)), | ||||
|             max_idle: 5, | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<C> Config<C> { | ||||
|     /// Set the `Connect` type to be used. | ||||
|     #[inline] | ||||
|     pub fn connector<CC: Connect>(self, val: CC) -> Config<CC> { | ||||
|         Config { | ||||
|             connect_timeout: self.connect_timeout, | ||||
|             //connect_timeout: self.connect_timeout, | ||||
|             connector: val, | ||||
|             keep_alive: self.keep_alive, | ||||
|             keep_alive_timeout: Some(Duration::from_secs(60 * 2)), | ||||
|             keep_alive_timeout: self.keep_alive_timeout, | ||||
|             max_idle: self.max_idle, | ||||
|             max_sockets: self.max_sockets, | ||||
|             dns_workers: self.dns_workers, | ||||
|         } | ||||
|     } | ||||
|  | ||||
| @@ -189,15 +289,7 @@ impl<C> Config<C> where C: Connect + Send + 'static { | ||||
|         self | ||||
|     } | ||||
|  | ||||
|     /// Set the max table size allocated for holding on to live sockets. | ||||
|     /// | ||||
|     /// Default is 1024. | ||||
|     #[inline] | ||||
|     pub fn max_sockets(mut self, val: usize) -> Config<C> { | ||||
|         self.max_sockets = val; | ||||
|         self | ||||
|     } | ||||
|  | ||||
|     /* | ||||
|     /// Set the timeout for connecting to a URL. | ||||
|     /// | ||||
|     /// Default is 10 seconds. | ||||
| @@ -206,584 +298,25 @@ impl<C> Config<C> where C: Connect + Send + 'static { | ||||
|         self.connect_timeout = val; | ||||
|         self | ||||
|     } | ||||
|     */ | ||||
| } | ||||
|  | ||||
|     /// Set number of Dns workers to use for this client | ||||
|     /// | ||||
|     /// Default is 4 | ||||
|     #[inline] | ||||
|     pub fn dns_workers(mut self, workers: usize) -> Config<C> { | ||||
|         self.dns_workers = workers; | ||||
|         self | ||||
|     } | ||||
|  | ||||
| impl<C: Connect> Config<C> { | ||||
|     /// Construct the Client with this configuration. | ||||
|     #[inline] | ||||
|     pub fn build<H: Handler<C::Output>>(self) -> ::Result<Client<H>> { | ||||
|         Client::configured(self) | ||||
|     pub fn build(self, handle: &Handle) -> Client<C> { | ||||
|         Client::configured(self, handle) | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Default for Config<DefaultConnector> { | ||||
|     fn default() -> Config<DefaultConnector> { | ||||
|         Config { | ||||
|             connect_timeout: Duration::from_secs(10), | ||||
|             connector: DefaultConnector::default(), | ||||
|             keep_alive: true, | ||||
|             keep_alive_timeout: Some(Duration::from_secs(60 * 2)), | ||||
|             max_idle: 5, | ||||
|             max_sockets: 1024, | ||||
|             dns_workers: 4, | ||||
|         } | ||||
| impl Config<UseDefaultConnector> { | ||||
|     /// Construct the Client with this configuration. | ||||
|     #[inline] | ||||
|     pub fn build(self, handle: &Handle) -> Client<HttpConnector> { | ||||
|         self.connector(HttpConnector::new(4, handle)).build(handle) | ||||
|     } | ||||
| } | ||||
|  | ||||
| /// An error that can occur when trying to queue a request. | ||||
| #[derive(Debug)] | ||||
| pub struct ClientError<H>(Option<(Url, H)>); | ||||
|  | ||||
| impl<H> ClientError<H> { | ||||
|     /// If the event loop was down, the `Url` and `Handler` can be recovered | ||||
|     /// from this method. | ||||
|     pub fn recover(self) -> Option<(Url, H)> { | ||||
|         self.0 | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<H: fmt::Debug + ::std::any::Any> ::std::error::Error for ClientError<H> { | ||||
|     fn description(&self) -> &str { | ||||
|         "Cannot queue request" | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<H> fmt::Display for ClientError<H> { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.write_str("Cannot queue request") | ||||
|     } | ||||
| } | ||||
|  | ||||
| /// A trait to react to client events that happen for each message. | ||||
| /// | ||||
| /// Each event handler returns it's desired `Next` action. | ||||
| pub trait Handler<T: Transport>: Send + 'static { | ||||
|     /// This event occurs first, triggering when a `Request` head can be written.. | ||||
|     fn on_request(&mut self, request: &mut Request) -> http::Next; | ||||
|     /// This event occurs each time the `Request` is ready to be written to. | ||||
|     fn on_request_writable(&mut self, request: &mut http::Encoder<T>) -> http::Next; | ||||
|     /// This event occurs after the first time this handler signals `Next::read()`, | ||||
|     /// and a Response has been parsed. | ||||
|     fn on_response(&mut self, response: Response) -> http::Next; | ||||
|     /// This event occurs each time the `Response` is ready to be read from. | ||||
|     fn on_response_readable(&mut self, response: &mut http::Decoder<T>) -> http::Next; | ||||
|  | ||||
|     /// This event occurs whenever an `Error` occurs outside of the other events. | ||||
|     /// | ||||
|     /// This could IO errors while waiting for events, or a timeout, etc. | ||||
|     fn on_error(&mut self, err: ::Error) -> http::Next { | ||||
|         debug!("default Handler.on_error({:?})", err); | ||||
|         http::Next::remove() | ||||
|     } | ||||
|  | ||||
|     /// This event occurs when this Handler has requested to remove the Transport. | ||||
|     fn on_remove(self, _transport: T) where Self: Sized { | ||||
|         debug!("default Handler.on_remove"); | ||||
|     } | ||||
|  | ||||
|     /// Receive a `Control` to manage waiting for this request. | ||||
|     fn on_control(&mut self, _: http::Control) { | ||||
|         debug!("default Handler.on_control()"); | ||||
|     } | ||||
| } | ||||
|  | ||||
| struct Message<H: Handler<T>, T: Transport> { | ||||
|     handler: H, | ||||
|     url: Option<Url>, | ||||
|     _marker: PhantomData<T>, | ||||
| } | ||||
|  | ||||
| impl<H: Handler<T>, T: Transport> http::MessageHandler<T> for Message<H, T> { | ||||
|     type Message = http::ClientMessage; | ||||
|  | ||||
|     fn on_outgoing(&mut self, head: &mut RequestHead) -> Next { | ||||
|         let url = self.url.take().expect("Message.url is missing"); | ||||
|         if let Some(host) = url.host_str() { | ||||
|             head.headers.set(Host { | ||||
|                 hostname: host.to_owned(), | ||||
|                 port: url.port(), | ||||
|             }); | ||||
|         } | ||||
|         head.subject.1 = RequestUri::AbsolutePath { | ||||
|             path: url.path().to_owned(), | ||||
|             query: url.query().map(|q| q.to_owned()), | ||||
|         }; | ||||
|         let mut req = self::request::new(head); | ||||
|         self.handler.on_request(&mut req) | ||||
|     } | ||||
|  | ||||
|     fn on_encode(&mut self, transport: &mut http::Encoder<T>) -> Next { | ||||
|         self.handler.on_request_writable(transport) | ||||
|     } | ||||
|  | ||||
|     fn on_incoming(&mut self, head: http::ResponseHead, _: &T) -> Next { | ||||
|         trace!("on_incoming {:?}", head); | ||||
|         let resp = response::new(head); | ||||
|         self.handler.on_response(resp) | ||||
|     } | ||||
|  | ||||
|     fn on_decode(&mut self, transport: &mut http::Decoder<T>) -> Next { | ||||
|         self.handler.on_response_readable(transport) | ||||
|     } | ||||
|  | ||||
|     fn on_error(&mut self, error: ::Error) -> Next { | ||||
|         self.handler.on_error(error) | ||||
|     } | ||||
|  | ||||
|     fn on_remove(self, transport: T) { | ||||
|         self.handler.on_remove(transport); | ||||
|     } | ||||
| } | ||||
|  | ||||
| struct Context<K, H, C: Connect> { | ||||
|     connect_timeout: Duration, | ||||
|     keep_alive: bool, | ||||
|     idle_conns: HashMap<K, VecDeque<http::Control>>, | ||||
|     queue: HashMap<K, VecDeque<Queued<H>>>, | ||||
|     awaiting_slot: VecDeque<(C::Key, C::Output)>, | ||||
| } | ||||
|  | ||||
| /// Macro for advancing state of a ClientFsm::Socket | ||||
| /// | ||||
| /// This was previously a method on Context, but due to eviction needs, this | ||||
| /// block now needs access to the registration APIs on rotor::Scope. | ||||
| macro_rules! conn_response { | ||||
|     ($scope:expr, $conn:expr, $time:expr) => {{ | ||||
|         match $conn { | ||||
|             Some((conn, timeout)) => { | ||||
|                 //TODO: HTTP2: a connection doesn't need to be idle to be used for a second stream | ||||
|                 if conn.is_idle() { | ||||
|                     $scope.idle_conns.entry(conn.key().clone()).or_insert_with(VecDeque::new) | ||||
|                         .push_back(conn.control()); | ||||
|                 } | ||||
|                 match timeout { | ||||
|                     Some(dur) => rotor::Response::ok(ClientFsm::Socket(conn)) | ||||
|                         .deadline($time + dur), | ||||
|                     None => rotor::Response::ok(ClientFsm::Socket(conn)), | ||||
|                 } | ||||
|  | ||||
|             } | ||||
|             None => { | ||||
|                 if let Some((key, socket)) = $scope.awaiting_slot.pop_front() { | ||||
|                     rotor_try!($scope.register(&socket, EventSet::writable() | EventSet::hup(), PollOpt::level())); | ||||
|                     rotor::Response::ok(ClientFsm::Connecting((key, socket))) | ||||
|                 } else { | ||||
|                     rotor::Response::done() | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
|     }} | ||||
| } | ||||
|  | ||||
| impl<K: http::Key, H, C: Connect> Context<K, H, C> { | ||||
|     fn pop_queue(&mut self, key: &K) -> Option<Queued<H>> { | ||||
|         let mut should_remove = false; | ||||
|         let queued = { | ||||
|             self.queue.get_mut(key).and_then(|vec| { | ||||
|                 let queued = vec.pop_front(); | ||||
|                 if vec.is_empty() { | ||||
|                     should_remove = true; | ||||
|                 } | ||||
|                 queued | ||||
|             }) | ||||
|         }; | ||||
|         if should_remove { | ||||
|             self.queue.remove(key); | ||||
|         } | ||||
|  | ||||
|         queued | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<K, H, T, C> http::MessageHandlerFactory<K, T> for Context<K, H, C> | ||||
|     where K: http::Key, | ||||
|           H: Handler<T>, | ||||
|           T: Transport, | ||||
|           C: Connect | ||||
| { | ||||
|     type Output = Message<H, T>; | ||||
|  | ||||
|     fn create(&mut self, seed: http::Seed<K>) -> Option<Self::Output> { | ||||
|         let key = seed.key(); | ||||
|         self.pop_queue(key).map(|queued| { | ||||
|             let (url, mut handler) = (queued.url, queued.handler); | ||||
|             handler.on_control(seed.control()); | ||||
|  | ||||
|             Message { | ||||
|                 handler: handler, | ||||
|                 url: Some(url), | ||||
|                 _marker: PhantomData, | ||||
|             } | ||||
|         }) | ||||
|     } | ||||
|  | ||||
|     fn keep_alive_interest(&self) -> Next { | ||||
|         Next::wait() | ||||
|     } | ||||
| } | ||||
|  | ||||
| enum Notify<T> { | ||||
|     Connect(Url, T), | ||||
|     Shutdown, | ||||
| } | ||||
|  | ||||
| enum ClientFsm<C, H> | ||||
| where C: Connect, | ||||
|       C::Output: Transport, | ||||
|       H: Handler<C::Output> { | ||||
|     Connector(C, http::channel::Receiver<Notify<H>>), | ||||
|     Connecting((C::Key, C::Output)), | ||||
|     Socket(http::Conn<C::Key, C::Output, Message<H, C::Output>>) | ||||
| } | ||||
|  | ||||
| unsafe impl<C, H> Send for ClientFsm<C, H> | ||||
| where | ||||
|     C: Connect + Send, | ||||
|     //C::Key, // Key doesn't need to be Send | ||||
|     C::Output: Transport, // Tranport doesn't need to be Send | ||||
|     H: Handler<C::Output> + Send | ||||
| {} | ||||
|  | ||||
| impl<C, H> rotor::Machine for ClientFsm<C, H> | ||||
| where C: Connect, | ||||
|       C::Key: fmt::Debug, | ||||
|       C::Output: Transport, | ||||
|       H: Handler<C::Output> { | ||||
|     type Context = Context<C::Key, H, C>; | ||||
|     type Seed = (C::Key, C::Output); | ||||
|  | ||||
|     fn create(seed: Self::Seed, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, rotor::Void> { | ||||
|         rotor_try!(scope.register(&seed.1, EventSet::writable() | EventSet::hup(), PollOpt::level())); | ||||
|         rotor::Response::ok(ClientFsm::Connecting(seed)) | ||||
|     } | ||||
|  | ||||
|     fn ready(self, events: EventSet, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> { | ||||
|         match self { | ||||
|             ClientFsm::Socket(conn) => { | ||||
|                 let mut conn = Some(conn); | ||||
|                 loop { | ||||
|                     match conn.take().unwrap().ready(events, scope) { | ||||
|                         ReadyResult::Done(res) => { | ||||
|                             let now = scope.now(); | ||||
|                             return conn_response!(scope, res, now); | ||||
|                         }, | ||||
|                         ReadyResult::Continue(c) => conn = Some(c), | ||||
|                     } | ||||
|                 } | ||||
|             }, | ||||
|             ClientFsm::Connecting(mut seed) => { | ||||
|                 if events.is_error() || events.is_hup() { | ||||
|                     if let Some(err) = seed.1.take_socket_error().err() { | ||||
|                         debug!("error while connecting: {:?}", err); | ||||
|                         scope.pop_queue(&seed.0).map(move |mut queued| queued.handler.on_error(::Error::Io(err))); | ||||
|                     } else { | ||||
|                         trace!("connecting is_error, but no socket error"); | ||||
|                     } | ||||
|  | ||||
|                     rotor::Response::done() | ||||
|                 } else if events.is_writable() { | ||||
|                     if scope.queue.contains_key(&seed.0) { | ||||
|                         trace!("connected and writable {:?}", seed.0); | ||||
|                         rotor::Response::ok( | ||||
|                             ClientFsm::Socket( | ||||
|                                 http::Conn::new( | ||||
|                                     seed.0, | ||||
|                                     seed.1, | ||||
|                                     Next::write().timeout(scope.connect_timeout), | ||||
|                                     scope.notifier(), | ||||
|                                     scope.now() | ||||
|                                 ).keep_alive(scope.keep_alive) | ||||
|                             ) | ||||
|                         ) | ||||
|                     } else { | ||||
|                         trace!("connected, but queued handler is gone: {:?}", seed.0); // probably took too long connecting | ||||
|                         rotor::Response::done() | ||||
|                     } | ||||
|                 } else { | ||||
|                     // spurious? | ||||
|                     rotor::Response::ok(ClientFsm::Connecting(seed)) | ||||
|                 } | ||||
|             } | ||||
|             ClientFsm::Connector(..) => { | ||||
|                 unreachable!("Connector can never be ready") | ||||
|             }, | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn spawned(self, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> { | ||||
|         match self { | ||||
|             ClientFsm::Connector(..) => self.connect(scope), | ||||
|             other => rotor::Response::ok(other) | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn spawn_error( | ||||
|         self, | ||||
|         scope: &mut Scope<Self::Context>, | ||||
|         error: rotor::SpawnError<Self::Seed> | ||||
|     ) -> rotor::Response<Self, Self::Seed> { | ||||
|         // see if there's an idle connections that can be terminated. If yes, put this seed on a | ||||
|         // list waiting for empty slot. | ||||
|         if let rotor::SpawnError::NoSlabSpace((key, socket)) = error { | ||||
|             if let Some(mut queued) = scope.pop_queue(&key) { | ||||
|                 trace!("attempting to remove an idle socket"); | ||||
|                 // Remove an idle connection. Any connection. Just make some space | ||||
|                 // for the new request. | ||||
|                 let mut remove_keys = Vec::new(); | ||||
|                 let mut found_idle = false; | ||||
|  | ||||
|                 // Check all idle connections regardless of origin | ||||
|                 for (key, idle) in scope.idle_conns.iter_mut() { | ||||
|                     // Pop from the front since those are lease recently used | ||||
|                     while let Some(ctrl) = idle.pop_front() { | ||||
|                         // Signal connection to close. An err here means the | ||||
|                         // socket is already dead can should be tossed. | ||||
|                         if ctrl.ready(Next::remove()).is_ok() { | ||||
|                             found_idle = true; | ||||
|                             break; | ||||
|                         } | ||||
|                     } | ||||
|  | ||||
|                     // This list is empty, mark it for removal | ||||
|                     if idle.is_empty() { | ||||
|                         remove_keys.push(key.to_owned()); | ||||
|                     } | ||||
|  | ||||
|                     // if found, stop looking for an idle connection. | ||||
|                     if found_idle { | ||||
|                         break; | ||||
|                     } | ||||
|                 } | ||||
|  | ||||
|                 trace!("idle conns: {:?}", scope.idle_conns); | ||||
|  | ||||
|                 // Remove empty idle lists. | ||||
|                 for key in &remove_keys { | ||||
|                     scope.idle_conns.remove(&key); | ||||
|                 } | ||||
|  | ||||
|                 if found_idle { | ||||
|                     // A socket should be evicted soon; put it on a queue to | ||||
|                     // consume newly freed slot. Also need to put the Queued<H> | ||||
|                     // back onto front of queue. | ||||
|                     scope.awaiting_slot.push_back((key.clone(), socket)); | ||||
|                     scope.queue | ||||
|                         .entry(key) | ||||
|                         .or_insert_with(VecDeque::new) | ||||
|                         .push_back(queued); | ||||
|                 } else { | ||||
|                     // Couldn't evict a socket, just run the error handler. | ||||
|                     debug!("Error spawning state machine; slab full and no sockets idle"); | ||||
|                     let _ = queued.handler.on_error(::Error::Full); | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         self.connect(scope) | ||||
|     } | ||||
|  | ||||
|     fn timeout(self, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> { | ||||
|         trace!("timeout now = {:?}", scope.now()); | ||||
|         match self { | ||||
|             ClientFsm::Connector(..) => { | ||||
|                 let now = scope.now(); | ||||
|                 let mut empty_keys = Vec::new(); | ||||
|                 { | ||||
|                     for (key, mut vec) in &mut scope.queue { | ||||
|                         while !vec.is_empty() && vec[0].deadline <= now { | ||||
|                             vec.pop_front() | ||||
|                                .map(|mut queued| queued.handler.on_error(::Error::Timeout)); | ||||
|                         } | ||||
|                         if vec.is_empty() { | ||||
|                             empty_keys.push(key.clone()); | ||||
|                         } | ||||
|                     } | ||||
|                 } | ||||
|                 for key in &empty_keys { | ||||
|                     scope.queue.remove(key); | ||||
|                 } | ||||
|                 match self.deadline(scope) { | ||||
|                     Some(deadline) => { | ||||
|                         rotor::Response::ok(self).deadline(deadline) | ||||
|                     }, | ||||
|                     None => rotor::Response::ok(self) | ||||
|                 } | ||||
|             } | ||||
|             ClientFsm::Connecting(..) => unreachable!(), | ||||
|             ClientFsm::Socket(conn) => { | ||||
|                 let res = conn.timeout(scope); | ||||
|                 let now = scope.now(); | ||||
|                 conn_response!(scope, res, now) | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn wakeup(self, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> { | ||||
|         match self { | ||||
|             ClientFsm::Connector(..) => { | ||||
|                 self.connect(scope) | ||||
|             }, | ||||
|             ClientFsm::Socket(conn) => { | ||||
|                 let res = conn.wakeup(scope); | ||||
|                 let now = scope.now(); | ||||
|                 conn_response!(scope, res, now) | ||||
|             }, | ||||
|             ClientFsm::Connecting(..) => unreachable!("connecting sockets should not be woken up") | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<C, H> ClientFsm<C, H> | ||||
| where C: Connect, | ||||
|       C::Key: fmt::Debug, | ||||
|       C::Output: Transport, | ||||
|       H: Handler<C::Output> { | ||||
|     fn connect(self, scope: &mut rotor::Scope<<Self as rotor::Machine>::Context>) -> rotor::Response<Self, <Self as rotor::Machine>::Seed> { | ||||
|         match self { | ||||
|             ClientFsm::Connector(mut connector, rx) => { | ||||
|                 if let Some((key, res)) = connector.connected() { | ||||
|                     match res { | ||||
|                         Ok(socket) => { | ||||
|                             trace!("connecting {:?}", key); | ||||
|                             return rotor::Response::spawn(ClientFsm::Connector(connector, rx), (key, socket)); | ||||
|                         }, | ||||
|                         Err(e) => { | ||||
|                             trace!("connect error = {:?}", e); | ||||
|                             scope.pop_queue(&key).map(|mut queued| queued.handler.on_error(::Error::Io(e))); | ||||
|                         } | ||||
|                     } | ||||
|                 } | ||||
|                 loop { | ||||
|                     match rx.try_recv() { | ||||
|                         Ok(Notify::Connect(url, mut handler)) => { | ||||
|                             // check pool for sockets to this domain | ||||
|                             if let Some(key) = connector.key(&url) { | ||||
|                                 let mut remove_idle = false; | ||||
|                                 let mut woke_up = false; | ||||
|                                 if let Some(mut idle) = scope.idle_conns.get_mut(&key) { | ||||
|                                     // Pop from back since those are most recently used. Connections | ||||
|                                     // at the front are allowed to expire. | ||||
|                                     while let Some(ctrl) = idle.pop_back() { | ||||
|                                         // err means the socket has since died | ||||
|                                         if ctrl.ready(Next::write()).is_ok() { | ||||
|                                             woke_up = true; | ||||
|                                             break; | ||||
|                                         } | ||||
|                                     } | ||||
|                                     remove_idle = idle.is_empty(); | ||||
|                                 } | ||||
|                                 if remove_idle { | ||||
|                                     scope.idle_conns.remove(&key); | ||||
|                                 } | ||||
|  | ||||
|                                 if woke_up { | ||||
|                                     trace!("woke up idle conn for '{}'", url); | ||||
|                                     let deadline = scope.now() + scope.connect_timeout; | ||||
|                                     scope.queue | ||||
|                                         .entry(key) | ||||
|                                         .or_insert_with(VecDeque::new) | ||||
|                                         .push_back(Queued { | ||||
|                                             deadline: deadline, | ||||
|                                             handler: handler, | ||||
|                                             url: url | ||||
|                                         }); | ||||
|                                     continue; | ||||
|                                 } | ||||
|                             } else { | ||||
|                                 // this connector cannot handle this url anyways | ||||
|                                 let _ = handler.on_error(io::Error::new(io::ErrorKind::InvalidInput, "invalid url for connector").into()); | ||||
|                                 continue; | ||||
|                             } | ||||
|                             // no exist connection, call connector | ||||
|                             match connector.connect(&url) { | ||||
|                                 Ok(key) => { | ||||
|                                     let deadline = scope.now() + scope.connect_timeout; | ||||
|                                     scope.queue | ||||
|                                         .entry(key) | ||||
|                                         .or_insert_with(VecDeque::new) | ||||
|                                         .push_back(Queued { | ||||
|                                             deadline: deadline, | ||||
|                                             handler: handler, | ||||
|                                             url: url | ||||
|                                         }); | ||||
|                                 } | ||||
|                                 Err(e) => { | ||||
|                                     let _todo = handler.on_error(e.into()); | ||||
|                                     trace!("Connect error, next={:?}", _todo); | ||||
|                                     continue; | ||||
|                                 } | ||||
|                             } | ||||
|                         } | ||||
|                         Ok(Notify::Shutdown) => { | ||||
|                             scope.shutdown_loop(); | ||||
|                             return rotor::Response::done() | ||||
|                         }, | ||||
|                         Err(mpsc::TryRecvError::Disconnected) => { | ||||
|                             // if there is no way to send additional requests, | ||||
|                             // what more can the loop do? i suppose we should | ||||
|                             // shutdown. | ||||
|                             scope.shutdown_loop(); | ||||
|                             return rotor::Response::done() | ||||
|                         } | ||||
|                         Err(mpsc::TryRecvError::Empty) => { | ||||
|                             // spurious wakeup or loop is done | ||||
|                             let fsm = ClientFsm::Connector(connector, rx); | ||||
|                             return match fsm.deadline(scope) { | ||||
|                                 Some(deadline) => { | ||||
|                                     rotor::Response::ok(fsm).deadline(deadline) | ||||
|                                 }, | ||||
|                                 None => rotor::Response::ok(fsm) | ||||
|                             }; | ||||
|                         } | ||||
|                     } | ||||
|                 } | ||||
|             }, | ||||
|             other => rotor::Response::ok(other) | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn deadline(&self, scope: &mut rotor::Scope<<Self as rotor::Machine>::Context>) -> Option<rotor::Time> { | ||||
|         match *self { | ||||
|             ClientFsm::Connector(..) => { | ||||
|                 let mut earliest = None; | ||||
|                 for vec in scope.queue.values() { | ||||
|                     for queued in vec { | ||||
|                         match earliest { | ||||
|                             Some(ref mut earliest) => { | ||||
|                                 if queued.deadline < *earliest { | ||||
|                                     *earliest = queued.deadline; | ||||
|                                 } | ||||
|                             } | ||||
|                             None => earliest = Some(queued.deadline) | ||||
|                         } | ||||
|                     } | ||||
|                 } | ||||
|                 trace!("deadline = {:?}, now = {:?}", earliest, scope.now()); | ||||
|                 earliest | ||||
|             } | ||||
|             _ => None | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| struct Queued<H> { | ||||
|     deadline: rotor::Time, | ||||
|     handler: H, | ||||
|     url: Url, | ||||
| } | ||||
|  | ||||
| #[doc(hidden)] | ||||
| #[allow(missing_debug_implementations)] | ||||
| pub struct Registration { | ||||
|     notify: (http::channel::Sender<self::dns::Answer>, http::channel::Receiver<self::dns::Answer>), | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod tests { | ||||
|   | ||||
							
								
								
									
										353
									
								
								src/client/pool.rs
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										353
									
								
								src/client/pool.rs
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,353 @@ | ||||
| use std::cell::{Cell, RefCell}; | ||||
| use std::collections::{HashMap, VecDeque}; | ||||
| use std::fmt; | ||||
| use std::io; | ||||
| use std::ops::{Deref, DerefMut, BitAndAssign}; | ||||
| use std::rc::Rc; | ||||
| use std::time::{Duration, Instant}; | ||||
|  | ||||
| use futures::{Future, Async, Poll}; | ||||
| use relay; | ||||
|  | ||||
| use http::{KeepAlive, KA}; | ||||
|  | ||||
| pub struct Pool<T> { | ||||
|     inner: Rc<RefCell<PoolInner<T>>>, | ||||
| } | ||||
|  | ||||
| struct PoolInner<T> { | ||||
|     enabled: bool, | ||||
|     idle: HashMap<Rc<String>, Vec<Entry<T>>>, | ||||
|     parked: HashMap<Rc<String>, VecDeque<relay::Sender<Entry<T>>>>, | ||||
|     timeout: Option<Duration>, | ||||
| } | ||||
|  | ||||
| impl<T: Clone> Pool<T> { | ||||
|     pub fn new(enabled: bool, timeout: Option<Duration>) -> Pool<T> { | ||||
|         Pool { | ||||
|             inner: Rc::new(RefCell::new(PoolInner { | ||||
|                 enabled: enabled, | ||||
|                 idle: HashMap::new(), | ||||
|                 parked: HashMap::new(), | ||||
|                 timeout: timeout, | ||||
|             })), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub fn checkout(&self, key: &str) -> Checkout<T> { | ||||
|         Checkout { | ||||
|             key: Rc::new(key.to_owned()), | ||||
|             pool: self.clone(), | ||||
|             parked: None, | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn put(&mut self, key: Rc<String>, entry: Entry<T>) { | ||||
|         trace!("Pool::put {:?}", key); | ||||
|         let mut remove_parked = false; | ||||
|         let tx = self.inner.borrow_mut().parked.get_mut(&key).and_then(|parked| { | ||||
|             let mut ret = None; | ||||
|             while let Some(tx) = parked.pop_front() { | ||||
|                 if !tx.is_canceled() { | ||||
|                     ret = Some(tx); | ||||
|                     break; | ||||
|                 } | ||||
|                 trace!("Pool::put removing canceled parked {:?}", key); | ||||
|             } | ||||
|             remove_parked = parked.is_empty(); | ||||
|             ret | ||||
|         }); | ||||
|         if remove_parked { | ||||
|             self.inner.borrow_mut().parked.remove(&key); | ||||
|         } | ||||
|  | ||||
|         if let Some(tx) = tx { | ||||
|             trace!("Pool::put found parked {:?}", key); | ||||
|             tx.complete(entry); | ||||
|         } else { | ||||
|             self.inner.borrow_mut() | ||||
|                 .idle.entry(key) | ||||
|                 .or_insert(Vec::new()) | ||||
|                 .push(entry); | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub fn pooled(&self, key: Rc<String>, value: T) -> Pooled<T> { | ||||
|         trace!("Pool::pooled {:?}", key); | ||||
|         Pooled { | ||||
|             entry: Entry { | ||||
|                 value: value, | ||||
|                 is_reused: false, | ||||
|                 status: Rc::new(Cell::new(KA::Busy)), | ||||
|             }, | ||||
|             key: key, | ||||
|             pool: self.clone(), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn is_enabled(&self) -> bool { | ||||
|         self.inner.borrow().enabled | ||||
|     } | ||||
|  | ||||
|     fn reuse(&self, key: Rc<String>, mut entry: Entry<T>) -> Pooled<T> { | ||||
|         trace!("Pool::reuse {:?}", key); | ||||
|         entry.is_reused = true; | ||||
|         entry.status.set(KA::Busy); | ||||
|         Pooled { | ||||
|             entry: entry, | ||||
|             key: key, | ||||
|             pool: self.clone(), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn park(&mut self, key: Rc<String>, tx: relay::Sender<Entry<T>>) { | ||||
|         trace!("Pool::park {:?}", key); | ||||
|         self.inner.borrow_mut() | ||||
|             .parked.entry(key) | ||||
|             .or_insert(VecDeque::new()) | ||||
|             .push_back(tx); | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<T> Clone for Pool<T> { | ||||
|     fn clone(&self) -> Pool<T> { | ||||
|         Pool { | ||||
|             inner: self.inner.clone(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Clone)] | ||||
| pub struct Pooled<T> { | ||||
|     entry: Entry<T>, | ||||
|     key: Rc<String>, | ||||
|     pool: Pool<T>, | ||||
| } | ||||
|  | ||||
| impl<T> Deref for Pooled<T> { | ||||
|     type Target = T; | ||||
|     fn deref(&self) -> &T { | ||||
|         &self.entry.value | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<T> DerefMut for Pooled<T> { | ||||
|     fn deref_mut(&mut self) -> &mut T { | ||||
|         &mut self.entry.value | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<T: Clone> KeepAlive for Pooled<T> { | ||||
|     fn busy(&mut self) { | ||||
|         self.entry.status.set(KA::Busy); | ||||
|     } | ||||
|  | ||||
|     fn disable(&mut self) { | ||||
|         self.entry.status.set(KA::Disabled); | ||||
|     } | ||||
|  | ||||
|     fn idle(&mut self) { | ||||
|         let previous = self.status(); | ||||
|         self.entry.status.set(KA::Idle(Instant::now())); | ||||
|         if let KA::Idle(..) = previous { | ||||
|             trace!("Pooled::idle already idle"); | ||||
|             return; | ||||
|         } | ||||
|         self.entry.is_reused = true; | ||||
|         if self.pool.is_enabled() { | ||||
|             self.pool.put(self.key.clone(), self.entry.clone()); | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn status(&self) -> KA { | ||||
|         self.entry.status.get() | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<T> fmt::Debug for Pooled<T> { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.debug_struct("Pooled") | ||||
|             .field("status", &self.entry.status.get()) | ||||
|             .field("key", &self.key) | ||||
|             .finish() | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<T: Clone> BitAndAssign<bool> for Pooled<T> { | ||||
|     fn bitand_assign(&mut self, enabled: bool) { | ||||
|         if !enabled { | ||||
|             self.disable(); | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Clone)] | ||||
| struct Entry<T> { | ||||
|     value: T, | ||||
|     is_reused: bool, | ||||
|     status: Rc<Cell<KA>>, | ||||
| } | ||||
|  | ||||
| pub struct Checkout<T> { | ||||
|     key: Rc<String>, | ||||
|     pool: Pool<T>, | ||||
|     parked: Option<relay::Receiver<Entry<T>>>, | ||||
| } | ||||
|  | ||||
| impl<T: Clone> Future for Checkout<T> { | ||||
|     type Item = Pooled<T>; | ||||
|     type Error = io::Error; | ||||
|  | ||||
|     fn poll(&mut self) -> Poll<Self::Item, Self::Error> { | ||||
|         trace!("Checkout::poll"); | ||||
|         let mut drop_parked = false; | ||||
|         if let Some(ref mut rx) = self.parked { | ||||
|             match rx.poll() { | ||||
|                 Ok(Async::Ready(entry)) => { | ||||
|                     trace!("Checkout::poll found client in relay for {:?}", self.key); | ||||
|                     return Ok(Async::Ready(self.pool.reuse(self.key.clone(), entry))); | ||||
|                 }, | ||||
|                 Ok(Async::NotReady) => (), | ||||
|                 Err(_canceled) => drop_parked = true, | ||||
|             } | ||||
|         } | ||||
|         if drop_parked { | ||||
|             self.parked.take(); | ||||
|         } | ||||
|         let expiration = Expiration::new(self.pool.inner.borrow().timeout); | ||||
|         let key = &self.key; | ||||
|         trace!("Checkout::poll url = {:?}, expiration = {:?}", key, expiration.0); | ||||
|         let mut should_remove = false; | ||||
|         let entry = self.pool.inner.borrow_mut().idle.get_mut(key).and_then(|list| { | ||||
|             trace!("Checkout::poll key found {:?}", key); | ||||
|             while let Some(entry) = list.pop() { | ||||
|                 match entry.status.get() { | ||||
|                     KA::Idle(idle_at) if !expiration.expires(idle_at) => { | ||||
|                         trace!("Checkout::poll found idle client for {:?}", key); | ||||
|                         should_remove = list.is_empty(); | ||||
|                         return Some(entry); | ||||
|                     }, | ||||
|                     _ => { | ||||
|                         trace!("Checkout::poll removing unacceptable pooled {:?}", key); | ||||
|                         // every other case the Entry should just be dropped | ||||
|                         // 1. Idle but expired | ||||
|                         // 2. Busy (something else somehow took it?) | ||||
|                         // 3. Disabled don't reuse of course | ||||
|                     } | ||||
|                 } | ||||
|             } | ||||
|             should_remove = true; | ||||
|             None | ||||
|         }); | ||||
|  | ||||
|         if should_remove { | ||||
|             self.pool.inner.borrow_mut().idle.remove(key); | ||||
|         } | ||||
|         match entry { | ||||
|             Some(entry) => Ok(Async::Ready(self.pool.reuse(self.key.clone(), entry))), | ||||
|             None => { | ||||
|                 if self.parked.is_none() { | ||||
|                     let (tx, mut rx) = relay::channel(); | ||||
|                     let _ = rx.poll(); // park this task | ||||
|                     self.pool.park(self.key.clone(), tx); | ||||
|                     self.parked = Some(rx); | ||||
|                 } | ||||
|                 Ok(Async::NotReady) | ||||
|             }, | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| struct Expiration(Option<Instant>); | ||||
|  | ||||
| impl Expiration { | ||||
|     fn new(dur: Option<Duration>) -> Expiration { | ||||
|         Expiration(dur.map(|dur| Instant::now() - dur)) | ||||
|     } | ||||
|  | ||||
|     fn expires(&self, instant: Instant) -> bool { | ||||
|         match self.0 { | ||||
|             Some(expire) => expire > instant, | ||||
|             None => false, | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod tests { | ||||
|     use std::rc::Rc; | ||||
|     use std::time::Duration; | ||||
|     use futures::{Async, Future}; | ||||
|     use http::KeepAlive; | ||||
|     use super::Pool; | ||||
|  | ||||
|     #[test] | ||||
|     fn test_pool_checkout_smoke() { | ||||
|         let pool = Pool::new(true, Some(Duration::from_secs(5))); | ||||
|         let key = Rc::new("foo".to_string()); | ||||
|         let mut pooled = pool.pooled(key.clone(), 41); | ||||
|         pooled.idle(); | ||||
|  | ||||
|         match pool.checkout(&key).poll().unwrap() { | ||||
|             Async::Ready(pooled) => assert_eq!(*pooled, 41), | ||||
|             _ => panic!("not ready"), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     #[test] | ||||
|     fn test_pool_checkout_returns_none_if_expired() { | ||||
|         ::futures::lazy(|| { | ||||
|             let pool = Pool::new(true, Some(Duration::from_secs(1))); | ||||
|             let key = Rc::new("foo".to_string()); | ||||
|             let mut pooled = pool.pooled(key.clone(), 41); | ||||
|             pooled.idle(); | ||||
|             ::std::thread::sleep(pool.inner.borrow().timeout.unwrap()); | ||||
|             assert!(pool.checkout(&key).poll().unwrap().is_not_ready()); | ||||
|             ::futures::future::ok::<(), ()>(()) | ||||
|         }).wait().unwrap(); | ||||
|     } | ||||
|  | ||||
|     #[test] | ||||
|     fn test_pool_removes_expired() { | ||||
|         let pool = Pool::new(true, Some(Duration::from_secs(1))); | ||||
|         let key = Rc::new("foo".to_string()); | ||||
|  | ||||
|         let mut pooled1 = pool.pooled(key.clone(), 41); | ||||
|         pooled1.idle(); | ||||
|         let mut pooled2 = pool.pooled(key.clone(), 5); | ||||
|         pooled2.idle(); | ||||
|         let mut pooled3 = pool.pooled(key.clone(), 99); | ||||
|         pooled3.idle(); | ||||
|  | ||||
|  | ||||
|         assert_eq!(pool.inner.borrow().idle.get(&key).map(|entries| entries.len()), Some(3)); | ||||
|         ::std::thread::sleep(pool.inner.borrow().timeout.unwrap()); | ||||
|  | ||||
|         pooled1.idle(); | ||||
|         pooled2.idle(); // idle after sleep, not expired | ||||
|         pool.checkout(&key).poll().unwrap(); | ||||
|         assert_eq!(pool.inner.borrow().idle.get(&key).map(|entries| entries.len()), Some(1)); | ||||
|         pool.checkout(&key).poll().unwrap(); | ||||
|         assert!(pool.inner.borrow().idle.get(&key).is_none()); | ||||
|     } | ||||
|  | ||||
|     #[test] | ||||
|     fn test_pool_checkout_task_unparked() { | ||||
|         let pool = Pool::new(true, Some(Duration::from_secs(10))); | ||||
|         let key = Rc::new("foo".to_string()); | ||||
|         let pooled1 = pool.pooled(key.clone(), 41); | ||||
|  | ||||
|         let mut pooled = pooled1.clone(); | ||||
|         let checkout = pool.checkout(&key).join(::futures::lazy(move || { | ||||
|             // the checkout future will park first, | ||||
|             // and then this lazy future will be polled, which will insert | ||||
|             // the pooled back into the pool | ||||
|             // | ||||
|             // this test makes sure that doing so will unpark the checkout | ||||
|             pooled.idle(); | ||||
|             Ok(()) | ||||
|         })).map(|(entry, _)| entry); | ||||
|         assert_eq!(*checkout.wait().unwrap(), *pooled1); | ||||
|     } | ||||
| } | ||||
| @@ -1,55 +1,90 @@ | ||||
| //! Client Requests | ||||
| use std::fmt; | ||||
|  | ||||
| use Url; | ||||
|  | ||||
| use header::Headers; | ||||
| use http::RequestHead; | ||||
| use http::{Body, RequestHead}; | ||||
| use method::Method; | ||||
| use uri::RequestUri; | ||||
| use version::HttpVersion; | ||||
|  | ||||
|  | ||||
|  | ||||
| /// A client request to a remote server. | ||||
| #[derive(Debug)] | ||||
| pub struct Request<'a> { | ||||
|     head: &'a mut RequestHead | ||||
| pub struct Request { | ||||
|     method: Method, | ||||
|     url: Url, | ||||
|     version: HttpVersion, | ||||
|     headers: Headers, | ||||
|     body: Option<Body>, | ||||
| } | ||||
|  | ||||
| impl<'a> Request<'a> { | ||||
| impl Request { | ||||
|     /// Construct a new Request. | ||||
|     #[inline] | ||||
|     pub fn new(method: Method, url: Url) -> Request { | ||||
|         Request { | ||||
|             method: method, | ||||
|             url: url, | ||||
|             version: HttpVersion::default(), | ||||
|             headers: Headers::new(), | ||||
|             body: None, | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     /// Read the Request Url. | ||||
|     #[inline] | ||||
|     pub fn uri(&self) -> &RequestUri { &self.head.subject.1 } | ||||
|     pub fn url(&self) -> &Url { &self.url } | ||||
|  | ||||
|     /// Readthe Request Version. | ||||
|     #[inline] | ||||
|     pub fn version(&self) -> &HttpVersion { &self.head.version } | ||||
|     pub fn version(&self) -> &HttpVersion { &self.version } | ||||
|  | ||||
|     /// Read the Request headers. | ||||
|     #[inline] | ||||
|     pub fn headers(&self) -> &Headers { &self.head.headers } | ||||
|     pub fn headers(&self) -> &Headers { &self.headers } | ||||
|  | ||||
|     /// Read the Request method. | ||||
|     #[inline] | ||||
|     pub fn method(&self) -> &Method { &self.head.subject.0 } | ||||
|     pub fn method(&self) -> &Method { &self.method } | ||||
|  | ||||
|     /// Set the Method of this request. | ||||
|     #[inline] | ||||
|     pub fn set_method(&mut self, method: Method) { self.head.subject.0 = method; } | ||||
|     pub fn set_method(&mut self, method: Method) { self.method = method; } | ||||
|  | ||||
|     /// Get a mutable reference to the Request headers. | ||||
|     #[inline] | ||||
|     pub fn headers_mut(&mut self) -> &mut Headers { &mut self.head.headers } | ||||
|     pub fn headers_mut(&mut self) -> &mut Headers { &mut self.headers } | ||||
|  | ||||
|     /// Set the `RequestUri` of this request. | ||||
|     /// Set the `Url` of this request. | ||||
|     #[inline] | ||||
|     pub fn set_uri(&mut self, uri: RequestUri) { self.head.subject.1 = uri; } | ||||
|     pub fn set_url(&mut self, url: Url) { self.url = url; } | ||||
|  | ||||
|     /// Set the `HttpVersion` of this request. | ||||
|     #[inline] | ||||
|     pub fn set_version(&mut self, version: HttpVersion) { self.head.version = version; } | ||||
|     pub fn set_version(&mut self, version: HttpVersion) { self.version = version; } | ||||
|  | ||||
|     /// Set the body of the request. | ||||
|     #[inline] | ||||
|     pub fn set_body<T: Into<Body>>(&mut self, body: T) { self.body = Some(body.into()); } | ||||
| } | ||||
|  | ||||
| pub fn new(head: &mut RequestHead) -> Request { | ||||
|     Request { head: head } | ||||
| impl fmt::Debug for Request { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.debug_struct("Request") | ||||
|             .field("method", &self.method) | ||||
|             .field("url", &self.url) | ||||
|             .field("version", &self.version) | ||||
|             .field("headers", &self.headers) | ||||
|             .finish() | ||||
|     } | ||||
| } | ||||
|  | ||||
| pub fn split(req: Request) -> (RequestHead, Option<Body>) { | ||||
|     let head = RequestHead { | ||||
|         subject: ::http::RequestLine(req.method, RequestUri::AbsoluteUri(req.url)), | ||||
|         headers: req.headers, | ||||
|         version: req.version, | ||||
|     }; | ||||
|     (head, req.body) | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
|   | ||||
| @@ -1,11 +1,11 @@ | ||||
| //! Client Responses | ||||
| use std::fmt; | ||||
|  | ||||
| use header; | ||||
| //use net::NetworkStream; | ||||
| use http::{self, RawStatus}; | ||||
| use http::{self, RawStatus, Body}; | ||||
| use status; | ||||
| use version; | ||||
|  | ||||
| pub fn new(incoming: http::ResponseHead) -> Response { | ||||
| pub fn new(incoming: http::ResponseHead, body: Option<Body>) -> Response { | ||||
|     trace!("Response::new"); | ||||
|     let status = status::StatusCode::from_u16(incoming.subject.0); | ||||
|     debug!("version={:?}, status={:?}", incoming.version, status); | ||||
| @@ -16,17 +16,18 @@ pub fn new(incoming: http::ResponseHead) -> Response { | ||||
|         version: incoming.version, | ||||
|         headers: incoming.headers, | ||||
|         status_raw: incoming.subject, | ||||
|         body: body, | ||||
|     } | ||||
|  | ||||
| } | ||||
|  | ||||
| /// A response for a client request to a remote server. | ||||
| #[derive(Debug)] | ||||
| pub struct Response { | ||||
|     status: status::StatusCode, | ||||
|     headers: header::Headers, | ||||
|     version: version::HttpVersion, | ||||
|     status_raw: RawStatus, | ||||
|     body: Option<Body>, | ||||
| } | ||||
|  | ||||
| impl Response { | ||||
| @@ -42,170 +43,23 @@ impl Response { | ||||
|     #[inline] | ||||
|     pub fn status_raw(&self) -> &RawStatus { &self.status_raw } | ||||
|  | ||||
|     /// Get the final URL of this response. | ||||
|     #[inline] | ||||
|     //pub fn url(&self) -> &Url { &self.url } | ||||
|  | ||||
|     /// Get the HTTP version of this response from the server. | ||||
|     #[inline] | ||||
|     pub fn version(&self) -> &version::HttpVersion { &self.version } | ||||
| } | ||||
|  | ||||
| /* | ||||
| impl Drop for Response { | ||||
|     fn drop(&mut self) { | ||||
|         // if not drained, theres old bits in the Reader. we can't reuse this, | ||||
|         // since those old bits would end up in new Responses | ||||
|         // | ||||
|         // otherwise, the response has been drained. we should check that the | ||||
|         // server has agreed to keep the connection open | ||||
|         let is_drained = !self.message.has_body(); | ||||
|         trace!("Response.drop is_drained={}", is_drained); | ||||
|         if !(is_drained && http::should_keep_alive(self.version, &self.headers)) { | ||||
|             trace!("Response.drop closing connection"); | ||||
|             if let Err(e) = self.message.close_connection() { | ||||
|                 error!("Response.drop error closing connection: {}", e); | ||||
|             } | ||||
|         } | ||||
|     /// Take the `Body` of this response. | ||||
|     #[inline] | ||||
|     pub fn body(mut self) -> Body { | ||||
|         self.body.take().unwrap_or(Body::empty()) | ||||
|     } | ||||
| } | ||||
| */ | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod tests { | ||||
|     /* | ||||
|     use std::io::{self, Read}; | ||||
|  | ||||
|     use url::Url; | ||||
|  | ||||
|     use header::TransferEncoding; | ||||
|     use header::Encoding; | ||||
|     use http::HttpMessage; | ||||
|     use mock::MockStream; | ||||
|     use status; | ||||
|     use version; | ||||
|     use http::h1::Http11Message; | ||||
|  | ||||
|     use super::Response; | ||||
|  | ||||
|     fn read_to_string(mut r: Response) -> io::Result<String> { | ||||
|         let mut s = String::new(); | ||||
|         try!(r.read_to_string(&mut s)); | ||||
|         Ok(s) | ||||
| impl fmt::Debug for Response { | ||||
|     fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { | ||||
|         f.debug_struct("Response") | ||||
|             .field("status", &self.status) | ||||
|             .field("version", &self.version) | ||||
|             .field("headers", &self.headers) | ||||
|             .finish() | ||||
|     } | ||||
|  | ||||
|  | ||||
|     #[test] | ||||
|     fn test_into_inner() { | ||||
|         let message: Box<HttpMessage> = Box::new( | ||||
|             Http11Message::with_stream(Box::new(MockStream::new()))); | ||||
|         let message = message.downcast::<Http11Message>().ok().unwrap(); | ||||
|         let b = message.into_inner().downcast::<MockStream>().ok().unwrap(); | ||||
|         assert_eq!(b, Box::new(MockStream::new())); | ||||
|     } | ||||
|  | ||||
|     #[test] | ||||
|     fn test_parse_chunked_response() { | ||||
|         let stream = MockStream::with_input(b"\ | ||||
|             HTTP/1.1 200 OK\r\n\ | ||||
|             Transfer-Encoding: chunked\r\n\ | ||||
|             \r\n\ | ||||
|             1\r\n\ | ||||
|             q\r\n\ | ||||
|             2\r\n\ | ||||
|             we\r\n\ | ||||
|             2\r\n\ | ||||
|             rt\r\n\ | ||||
|             0\r\n\ | ||||
|             \r\n" | ||||
|         ); | ||||
|  | ||||
|         let url = Url::parse("http://hyper.rs").unwrap(); | ||||
|         let res = Response::new(url, Box::new(stream)).unwrap(); | ||||
|  | ||||
|         // The status line is correct? | ||||
|         assert_eq!(res.status, status::StatusCode::Ok); | ||||
|         assert_eq!(res.version, version::HttpVersion::Http11); | ||||
|         // The header is correct? | ||||
|         match res.headers.get::<TransferEncoding>() { | ||||
|             Some(encodings) => { | ||||
|                 assert_eq!(1, encodings.len()); | ||||
|                 assert_eq!(Encoding::Chunked, encodings[0]); | ||||
|             }, | ||||
|             None => panic!("Transfer-Encoding: chunked expected!"), | ||||
|         }; | ||||
|         // The body is correct? | ||||
|         assert_eq!(read_to_string(res).unwrap(), "qwert".to_owned()); | ||||
|     } | ||||
|  | ||||
|     /// Tests that when a chunk size is not a valid radix-16 number, an error | ||||
|     /// is returned. | ||||
|     #[test] | ||||
|     fn test_invalid_chunk_size_not_hex_digit() { | ||||
|         let stream = MockStream::with_input(b"\ | ||||
|             HTTP/1.1 200 OK\r\n\ | ||||
|             Transfer-Encoding: chunked\r\n\ | ||||
|             \r\n\ | ||||
|             X\r\n\ | ||||
|             1\r\n\ | ||||
|             0\r\n\ | ||||
|             \r\n" | ||||
|         ); | ||||
|  | ||||
|         let url = Url::parse("http://hyper.rs").unwrap(); | ||||
|         let res = Response::new(url, Box::new(stream)).unwrap(); | ||||
|  | ||||
|         assert!(read_to_string(res).is_err()); | ||||
|     } | ||||
|  | ||||
|     /// Tests that when a chunk size contains an invalid extension, an error is | ||||
|     /// returned. | ||||
|     #[test] | ||||
|     fn test_invalid_chunk_size_extension() { | ||||
|         let stream = MockStream::with_input(b"\ | ||||
|             HTTP/1.1 200 OK\r\n\ | ||||
|             Transfer-Encoding: chunked\r\n\ | ||||
|             \r\n\ | ||||
|             1 this is an invalid extension\r\n\ | ||||
|             1\r\n\ | ||||
|             0\r\n\ | ||||
|             \r\n" | ||||
|         ); | ||||
|  | ||||
|         let url = Url::parse("http://hyper.rs").unwrap(); | ||||
|         let res = Response::new(url, Box::new(stream)).unwrap(); | ||||
|  | ||||
|         assert!(read_to_string(res).is_err()); | ||||
|     } | ||||
|  | ||||
|     /// Tests that when a valid extension that contains a digit is appended to | ||||
|     /// the chunk size, the chunk is correctly read. | ||||
|     #[test] | ||||
|     fn test_chunk_size_with_extension() { | ||||
|         let stream = MockStream::with_input(b"\ | ||||
|             HTTP/1.1 200 OK\r\n\ | ||||
|             Transfer-Encoding: chunked\r\n\ | ||||
|             \r\n\ | ||||
|             1;this is an extension with a digit 1\r\n\ | ||||
|             1\r\n\ | ||||
|             0\r\n\ | ||||
|             \r\n" | ||||
|         ); | ||||
|  | ||||
|         let url = Url::parse("http://hyper.rs").unwrap(); | ||||
|         let res = Response::new(url, Box::new(stream)).unwrap(); | ||||
|  | ||||
|         assert_eq!(read_to_string(res).unwrap(), "1".to_owned()); | ||||
|     } | ||||
|  | ||||
|     #[test] | ||||
|     fn test_parse_error_closes() { | ||||
|         let url = Url::parse("http://hyper.rs").unwrap(); | ||||
|         let stream = MockStream::with_input(b"\ | ||||
|             definitely not http | ||||
|         "); | ||||
|  | ||||
|         assert!(Response::new(url, Box::new(stream)).is_err()); | ||||
|     } | ||||
|     */ | ||||
| } | ||||
|   | ||||
		Reference in New Issue
	
	Block a user