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:
Sean McArthur
2016-11-17 17:31:42 -08:00
parent e23689122a
commit 2d2d5574a6
43 changed files with 2775 additions and 5033 deletions

View File

@@ -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>();
}

View File

@@ -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 })
}

View File

@@ -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
View 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);
}
}

View File

@@ -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)]

View File

@@ -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());
}
*/
}