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