feat(lib): switch to non-blocking (asynchronous) IO
BREAKING CHANGE: This breaks a lot of the Client and Server APIs. Check the documentation for how Handlers can be used for asynchronous events.
This commit is contained in:
@@ -1,79 +0,0 @@
|
||||
use std::sync::{Arc, mpsc};
|
||||
use std::thread;
|
||||
|
||||
use net::NetworkListener;
|
||||
|
||||
pub struct ListenerPool<A: NetworkListener> {
|
||||
acceptor: A
|
||||
}
|
||||
|
||||
impl<A: NetworkListener + Send + 'static> ListenerPool<A> {
|
||||
/// Create a thread pool to manage the acceptor.
|
||||
pub fn new(acceptor: A) -> ListenerPool<A> {
|
||||
ListenerPool { acceptor: acceptor }
|
||||
}
|
||||
|
||||
/// Runs the acceptor pool. Blocks until the acceptors are closed.
|
||||
///
|
||||
/// ## Panics
|
||||
///
|
||||
/// Panics if threads == 0.
|
||||
pub fn accept<F>(self, work: F, threads: usize)
|
||||
where F: Fn(A::Stream) + Send + Sync + 'static {
|
||||
assert!(threads != 0, "Can't accept on 0 threads.");
|
||||
|
||||
let (super_tx, supervisor_rx) = mpsc::channel();
|
||||
|
||||
let work = Arc::new(work);
|
||||
|
||||
// Begin work.
|
||||
for _ in 0..threads {
|
||||
spawn_with(super_tx.clone(), work.clone(), self.acceptor.clone())
|
||||
}
|
||||
|
||||
// Monitor for panics.
|
||||
// FIXME(reem): This won't ever exit since we still have a super_tx handle.
|
||||
for _ in supervisor_rx.iter() {
|
||||
spawn_with(super_tx.clone(), work.clone(), self.acceptor.clone());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn spawn_with<A, F>(supervisor: mpsc::Sender<()>, work: Arc<F>, mut acceptor: A)
|
||||
where A: NetworkListener + Send + 'static,
|
||||
F: Fn(<A as NetworkListener>::Stream) + Send + Sync + 'static {
|
||||
thread::spawn(move || {
|
||||
let _sentinel = Sentinel::new(supervisor, ());
|
||||
|
||||
loop {
|
||||
match acceptor.accept() {
|
||||
Ok(stream) => work(stream),
|
||||
Err(e) => {
|
||||
error!("Connection failed: {}", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
struct Sentinel<T: Send + 'static> {
|
||||
value: Option<T>,
|
||||
supervisor: mpsc::Sender<T>,
|
||||
}
|
||||
|
||||
impl<T: Send + 'static> Sentinel<T> {
|
||||
fn new(channel: mpsc::Sender<T>, data: T) -> Sentinel<T> {
|
||||
Sentinel {
|
||||
value: Some(data),
|
||||
supervisor: channel,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Send + 'static> Drop for Sentinel<T> {
|
||||
fn drop(&mut self) {
|
||||
// Respawn ourselves
|
||||
let _ = self.supervisor.send(self.value.take().unwrap());
|
||||
}
|
||||
}
|
||||
|
||||
58
src/server/message.rs
Normal file
58
src/server/message.rs
Normal file
@@ -0,0 +1,58 @@
|
||||
use std::marker::PhantomData;
|
||||
|
||||
|
||||
use http::{self, Next};
|
||||
use net::Transport;
|
||||
|
||||
use super::{Handler, request, response};
|
||||
|
||||
/// A MessageHandler for a Server.
|
||||
///
|
||||
/// This should be really thin glue between http::MessageHandler and
|
||||
/// server::Handler, but largely just providing the proper types one
|
||||
/// would expect in a Server Handler.
|
||||
pub struct Message<H: Handler<T>, T: Transport> {
|
||||
handler: H,
|
||||
_marker: PhantomData<T>
|
||||
}
|
||||
|
||||
impl<H: Handler<T>, T: Transport> Message<H, T> {
|
||||
pub fn new(handler: H) -> Message<H, T> {
|
||||
Message {
|
||||
handler: handler,
|
||||
_marker: PhantomData,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<H: Handler<T>, T: Transport> http::MessageHandler<T> for Message<H, T> {
|
||||
type Message = http::ServerMessage;
|
||||
|
||||
fn on_incoming(&mut self, head: http::RequestHead) -> Next {
|
||||
trace!("on_incoming {:?}", head);
|
||||
let req = request::new(head);
|
||||
self.handler.on_request(req)
|
||||
}
|
||||
|
||||
fn on_decode(&mut self, transport: &mut http::Decoder<T>) -> Next {
|
||||
self.handler.on_request_readable(transport)
|
||||
}
|
||||
|
||||
fn on_outgoing(&mut self, head: &mut http::MessageHead<::status::StatusCode>) -> Next {
|
||||
let mut res = response::new(head);
|
||||
self.handler.on_response(&mut res)
|
||||
}
|
||||
|
||||
fn on_encode(&mut self, transport: &mut http::Encoder<T>) -> Next {
|
||||
self.handler.on_response_writable(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);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,510 +1,369 @@
|
||||
//! HTTP Server
|
||||
//!
|
||||
//! # Server
|
||||
//!
|
||||
//! A `Server` is created to listen on port, parse HTTP requests, and hand
|
||||
//! them off to a `Handler`. By default, the Server will listen across multiple
|
||||
//! threads, but that can be configured to a single thread if preferred.
|
||||
//!
|
||||
//! # Handling requests
|
||||
//!
|
||||
//! You must pass a `Handler` to the Server that will handle requests. There is
|
||||
//! a default implementation for `fn`s and closures, allowing you pass one of
|
||||
//! those easily.
|
||||
//!
|
||||
//!
|
||||
//! ```no_run
|
||||
//! use hyper::server::{Server, Request, Response};
|
||||
//!
|
||||
//! fn hello(req: Request, res: Response) {
|
||||
//! // handle things here
|
||||
//! }
|
||||
//!
|
||||
//! Server::http("0.0.0.0:0").unwrap().handle(hello).unwrap();
|
||||
//! ```
|
||||
//!
|
||||
//! As with any trait, you can also define a struct and implement `Handler`
|
||||
//! directly on your own type, and pass that to the `Server` instead.
|
||||
//!
|
||||
//! ```no_run
|
||||
//! use std::sync::Mutex;
|
||||
//! use std::sync::mpsc::{channel, Sender};
|
||||
//! use hyper::server::{Handler, Server, Request, Response};
|
||||
//!
|
||||
//! struct SenderHandler {
|
||||
//! sender: Mutex<Sender<&'static str>>
|
||||
//! }
|
||||
//!
|
||||
//! impl Handler for SenderHandler {
|
||||
//! fn handle(&self, req: Request, res: Response) {
|
||||
//! self.sender.lock().unwrap().send("start").unwrap();
|
||||
//! }
|
||||
//! }
|
||||
//!
|
||||
//!
|
||||
//! let (tx, rx) = channel();
|
||||
//! Server::http("0.0.0.0:0").unwrap().handle(SenderHandler {
|
||||
//! sender: Mutex::new(tx)
|
||||
//! }).unwrap();
|
||||
//! ```
|
||||
//!
|
||||
//! Since the `Server` will be listening on multiple threads, the `Handler`
|
||||
//! must implement `Sync`: any mutable state must be synchronized.
|
||||
//!
|
||||
//! ```no_run
|
||||
//! use std::sync::atomic::{AtomicUsize, Ordering};
|
||||
//! use hyper::server::{Server, Request, Response};
|
||||
//!
|
||||
//! let counter = AtomicUsize::new(0);
|
||||
//! Server::http("0.0.0.0:0").unwrap().handle(move |req: Request, res: Response| {
|
||||
//! counter.fetch_add(1, Ordering::Relaxed);
|
||||
//! }).unwrap();
|
||||
//! ```
|
||||
//!
|
||||
//! # The `Request` and `Response` pair
|
||||
//!
|
||||
//! A `Handler` receives a pair of arguments, a `Request` and a `Response`. The
|
||||
//! `Request` includes access to the `method`, `uri`, and `headers` of the
|
||||
//! incoming HTTP request. It also implements `std::io::Read`, in order to
|
||||
//! read any body, such as with `POST` or `PUT` messages.
|
||||
//!
|
||||
//! Likewise, the `Response` includes ways to set the `status` and `headers`,
|
||||
//! and implements `std::io::Write` to allow writing the response body.
|
||||
//!
|
||||
//! ```no_run
|
||||
//! use std::io;
|
||||
//! use hyper::server::{Server, Request, Response};
|
||||
//! use hyper::status::StatusCode;
|
||||
//!
|
||||
//! Server::http("0.0.0.0:0").unwrap().handle(|mut req: Request, mut res: Response| {
|
||||
//! match req.method {
|
||||
//! hyper::Post => {
|
||||
//! io::copy(&mut req, &mut res.start().unwrap()).unwrap();
|
||||
//! },
|
||||
//! _ => *res.status_mut() = StatusCode::MethodNotAllowed
|
||||
//! }
|
||||
//! }).unwrap();
|
||||
//! ```
|
||||
//!
|
||||
//! ## An aside: Write Status
|
||||
//!
|
||||
//! The `Response` uses a phantom type parameter to determine its write status.
|
||||
//! What does that mean? In short, it ensures you never write a body before
|
||||
//! adding all headers, and never add a header after writing some of the body.
|
||||
//!
|
||||
//! This is often done in most implementations by include a boolean property
|
||||
//! on the response, such as `headers_written`, checking that each time the
|
||||
//! body has something to write, so as to make sure the headers are sent once,
|
||||
//! and only once. But this has 2 downsides:
|
||||
//!
|
||||
//! 1. You are typically never notified that your late header is doing nothing.
|
||||
//! 2. There's a runtime cost to checking on every write.
|
||||
//!
|
||||
//! Instead, hyper handles this statically, or at compile-time. A
|
||||
//! `Response<Fresh>` includes a `headers_mut()` method, allowing you add more
|
||||
//! headers. It also does not implement `Write`, so you can't accidentally
|
||||
//! write early. Once the "head" of the response is correct, you can "send" it
|
||||
//! out by calling `start` on the `Response<Fresh>`. This will return a new
|
||||
//! `Response<Streaming>` object, that no longer has `headers_mut()`, but does
|
||||
//! implement `Write`.
|
||||
//! them off to a `Handler`.
|
||||
use std::fmt;
|
||||
use std::io::{self, ErrorKind, BufWriter, Write};
|
||||
use std::net::{SocketAddr, ToSocketAddrs};
|
||||
use std::thread::{self, JoinHandle};
|
||||
use std::net::SocketAddr;
|
||||
use std::sync::Arc;
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::time::Duration;
|
||||
|
||||
use num_cpus;
|
||||
use rotor::mio::{EventSet, PollOpt};
|
||||
use rotor::{self, Scope};
|
||||
|
||||
pub use self::request::Request;
|
||||
pub use self::response::Response;
|
||||
|
||||
pub use net::{Fresh, Streaming};
|
||||
use http::{self, Next};
|
||||
use net::{Accept, HttpListener, HttpsListener, SslServer, Transport};
|
||||
|
||||
use Error;
|
||||
use buffer::BufReader;
|
||||
use header::{Headers, Expect, Connection};
|
||||
use http;
|
||||
use method::Method;
|
||||
use net::{NetworkListener, NetworkStream, HttpListener, HttpsListener, Ssl};
|
||||
use status::StatusCode;
|
||||
use uri::RequestUri;
|
||||
use version::HttpVersion::Http11;
|
||||
|
||||
use self::listener::ListenerPool;
|
||||
mod request;
|
||||
mod response;
|
||||
mod message;
|
||||
|
||||
pub mod request;
|
||||
pub mod response;
|
||||
|
||||
mod listener;
|
||||
|
||||
/// A server can listen on a TCP socket.
|
||||
///
|
||||
/// Once listening, it will create a `Request`/`Response` pair for each
|
||||
/// incoming connection, and hand them to the provided handler.
|
||||
#[derive(Debug)]
|
||||
pub struct Server<L = HttpListener> {
|
||||
listener: L,
|
||||
timeouts: Timeouts,
|
||||
/// A configured `Server` ready to run.
|
||||
pub struct ServerLoop<A, H> where A: Accept, H: HandlerFactory<A::Output> {
|
||||
inner: Option<(rotor::Loop<ServerFsm<A, H>>, Context<H>)>,
|
||||
}
|
||||
|
||||
#[derive(Clone, Copy, Debug)]
|
||||
struct Timeouts {
|
||||
read: Option<Duration>,
|
||||
write: Option<Duration>,
|
||||
keep_alive: Option<Duration>,
|
||||
}
|
||||
|
||||
impl Default for Timeouts {
|
||||
fn default() -> Timeouts {
|
||||
Timeouts {
|
||||
read: None,
|
||||
write: None,
|
||||
keep_alive: Some(Duration::from_secs(5))
|
||||
}
|
||||
impl<A: Accept, H: HandlerFactory<A::Output>> fmt::Debug for ServerLoop<A, H> {
|
||||
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
|
||||
f.pad("ServerLoop")
|
||||
}
|
||||
}
|
||||
|
||||
macro_rules! try_option(
|
||||
($e:expr) => {{
|
||||
match $e {
|
||||
Some(v) => v,
|
||||
None => return None
|
||||
}
|
||||
}}
|
||||
);
|
||||
/// A Server that can accept incoming network requests.
|
||||
#[derive(Debug)]
|
||||
pub struct Server<T: Accept> {
|
||||
listener: T,
|
||||
keep_alive: bool,
|
||||
idle_timeout: Duration,
|
||||
max_sockets: usize,
|
||||
}
|
||||
|
||||
impl<L: NetworkListener> Server<L> {
|
||||
/// Creates a new server with the provided handler.
|
||||
impl<T> Server<T> where T: Accept, T::Output: Transport {
|
||||
/// Creates a new server with the provided Listener.
|
||||
#[inline]
|
||||
pub fn new(listener: L) -> Server<L> {
|
||||
pub fn new(listener: T) -> Server<T> {
|
||||
Server {
|
||||
listener: listener,
|
||||
timeouts: Timeouts::default()
|
||||
keep_alive: true,
|
||||
idle_timeout: Duration::from_secs(10),
|
||||
max_sockets: 4096,
|
||||
}
|
||||
}
|
||||
|
||||
/// Controls keep-alive for this server.
|
||||
/// Enables or disables HTTP keep-alive.
|
||||
///
|
||||
/// The timeout duration passed will be used to determine how long
|
||||
/// to keep the connection alive before dropping it.
|
||||
///
|
||||
/// Passing `None` will disable keep-alive.
|
||||
///
|
||||
/// Default is enabled with a 5 second timeout.
|
||||
#[inline]
|
||||
pub fn keep_alive(&mut self, timeout: Option<Duration>) {
|
||||
self.timeouts.keep_alive = timeout;
|
||||
/// Default is true.
|
||||
pub fn keep_alive(mut self, val: bool) -> Server<T> {
|
||||
self.keep_alive = val;
|
||||
self
|
||||
}
|
||||
|
||||
/// Sets the read timeout for all Request reads.
|
||||
pub fn set_read_timeout(&mut self, dur: Option<Duration>) {
|
||||
self.timeouts.read = dur;
|
||||
/// Sets how long an idle connection will be kept before closing.
|
||||
///
|
||||
/// Default is 10 seconds.
|
||||
pub fn idle_timeout(mut self, val: Duration) -> Server<T> {
|
||||
self.idle_timeout = val;
|
||||
self
|
||||
}
|
||||
|
||||
/// Sets the write timeout for all Response writes.
|
||||
pub fn set_write_timeout(&mut self, dur: Option<Duration>) {
|
||||
self.timeouts.write = dur;
|
||||
/// Sets the maximum open sockets for this Server.
|
||||
///
|
||||
/// Default is 4096, but most servers can handle much more than this.
|
||||
pub fn max_sockets(mut self, val: usize) -> Server<T> {
|
||||
self.max_sockets = val;
|
||||
self
|
||||
}
|
||||
}
|
||||
|
||||
impl Server<HttpListener> {
|
||||
/// Creates a new server that will handle `HttpStream`s.
|
||||
pub fn http<To: ToSocketAddrs>(addr: To) -> ::Result<Server<HttpListener>> {
|
||||
HttpListener::new(addr).map(Server::new)
|
||||
impl Server<HttpListener> { //<H: HandlerFactory<<HttpListener as Accept>::Output>> Server<HttpListener, H> {
|
||||
/// Creates a new HTTP server config listening on the provided address.
|
||||
pub fn http(addr: &SocketAddr) -> ::Result<Server<HttpListener>> {
|
||||
use ::rotor::mio::tcp::TcpListener;
|
||||
TcpListener::bind(addr)
|
||||
.map(HttpListener)
|
||||
.map(Server::new)
|
||||
.map_err(From::from)
|
||||
}
|
||||
}
|
||||
|
||||
impl<S: Ssl + Clone + Send> Server<HttpsListener<S>> {
|
||||
/// Creates a new server that will handle `HttpStream`s over SSL.
|
||||
|
||||
impl<S: SslServer> Server<HttpsListener<S>> {
|
||||
/// Creates a new server config that will handle `HttpStream`s over SSL.
|
||||
///
|
||||
/// You can use any SSL implementation, as long as implements `hyper::net::Ssl`.
|
||||
pub fn https<A: ToSocketAddrs>(addr: A, ssl: S) -> ::Result<Server<HttpsListener<S>>> {
|
||||
HttpsListener::new(addr, ssl).map(Server::new)
|
||||
pub fn https(addr: &SocketAddr, ssl: S) -> ::Result<Server<HttpsListener<S>>> {
|
||||
HttpsListener::new(addr, ssl)
|
||||
.map(Server::new)
|
||||
.map_err(From::from)
|
||||
}
|
||||
}
|
||||
|
||||
impl<L: NetworkListener + Send + 'static> Server<L> {
|
||||
|
||||
impl<A: Accept> Server<A> where A::Output: Transport {
|
||||
/// Binds to a socket and starts handling connections.
|
||||
pub fn handle<H: Handler + 'static>(self, handler: H) -> ::Result<Listening> {
|
||||
self.handle_threads(handler, num_cpus::get() * 5 / 4)
|
||||
}
|
||||
pub fn handle<H>(self, factory: H) -> ::Result<(Listening, ServerLoop<A, H>)>
|
||||
where H: HandlerFactory<A::Output> {
|
||||
let addr = try!(self.listener.local_addr());
|
||||
let shutdown = Arc::new(AtomicBool::new(false));
|
||||
let shutdown_rx = shutdown.clone();
|
||||
|
||||
/// Binds to a socket and starts handling connections with the provided
|
||||
/// number of threads.
|
||||
pub fn handle_threads<H: Handler + 'static>(self, handler: H,
|
||||
threads: usize) -> ::Result<Listening> {
|
||||
handle(self, handler, threads)
|
||||
}
|
||||
}
|
||||
|
||||
fn handle<H, L>(mut server: Server<L>, handler: H, threads: usize) -> ::Result<Listening>
|
||||
where H: Handler + 'static, L: NetworkListener + Send + 'static {
|
||||
let socket = try!(server.listener.local_addr());
|
||||
|
||||
debug!("threads = {:?}", threads);
|
||||
let pool = ListenerPool::new(server.listener);
|
||||
let worker = Worker::new(handler, server.timeouts);
|
||||
let work = move |mut stream| worker.handle_connection(&mut stream);
|
||||
|
||||
let guard = thread::spawn(move || pool.accept(work, threads));
|
||||
|
||||
Ok(Listening {
|
||||
_guard: Some(guard),
|
||||
socket: socket,
|
||||
})
|
||||
}
|
||||
|
||||
struct Worker<H: Handler + 'static> {
|
||||
handler: H,
|
||||
timeouts: Timeouts,
|
||||
}
|
||||
|
||||
impl<H: Handler + 'static> Worker<H> {
|
||||
fn new(handler: H, timeouts: Timeouts) -> Worker<H> {
|
||||
Worker {
|
||||
handler: handler,
|
||||
timeouts: timeouts,
|
||||
}
|
||||
}
|
||||
|
||||
fn handle_connection<S>(&self, mut stream: &mut S) where S: NetworkStream + Clone {
|
||||
debug!("Incoming stream");
|
||||
|
||||
self.handler.on_connection_start();
|
||||
|
||||
if let Err(e) = self.set_timeouts(&*stream) {
|
||||
error!("set_timeouts error: {:?}", e);
|
||||
return;
|
||||
}
|
||||
|
||||
let addr = match stream.peer_addr() {
|
||||
Ok(addr) => addr,
|
||||
Err(e) => {
|
||||
error!("Peer Name error: {:?}", e);
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let stream_clone: &mut NetworkStream = &mut stream.clone();
|
||||
let mut rdr = BufReader::new(stream_clone);
|
||||
let mut wrt = BufWriter::new(stream);
|
||||
|
||||
while self.keep_alive_loop(&mut rdr, &mut wrt, addr) {
|
||||
if let Err(e) = self.set_read_timeout(*rdr.get_ref(), self.timeouts.keep_alive) {
|
||||
error!("set_read_timeout keep_alive {:?}", e);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
self.handler.on_connection_end();
|
||||
|
||||
debug!("keep_alive loop ending for {}", addr);
|
||||
}
|
||||
|
||||
fn set_timeouts(&self, s: &NetworkStream) -> io::Result<()> {
|
||||
try!(self.set_read_timeout(s, self.timeouts.read));
|
||||
self.set_write_timeout(s, self.timeouts.write)
|
||||
}
|
||||
|
||||
fn set_write_timeout(&self, s: &NetworkStream, timeout: Option<Duration>) -> io::Result<()> {
|
||||
s.set_write_timeout(timeout)
|
||||
}
|
||||
|
||||
fn set_read_timeout(&self, s: &NetworkStream, timeout: Option<Duration>) -> io::Result<()> {
|
||||
s.set_read_timeout(timeout)
|
||||
}
|
||||
|
||||
fn keep_alive_loop<W: Write>(&self, mut rdr: &mut BufReader<&mut NetworkStream>,
|
||||
wrt: &mut W, addr: SocketAddr) -> bool {
|
||||
let req = match Request::new(rdr, addr) {
|
||||
Ok(req) => req,
|
||||
Err(Error::Io(ref e)) if e.kind() == ErrorKind::ConnectionAborted => {
|
||||
trace!("tcp closed, cancelling keep-alive loop");
|
||||
return false;
|
||||
}
|
||||
Err(Error::Io(e)) => {
|
||||
debug!("ioerror in keepalive loop = {:?}", e);
|
||||
return false;
|
||||
}
|
||||
Err(e) => {
|
||||
//TODO: send a 400 response
|
||||
error!("request error = {:?}", e);
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
if !self.handle_expect(&req, wrt) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if let Err(e) = req.set_read_timeout(self.timeouts.read) {
|
||||
error!("set_read_timeout {:?}", e);
|
||||
return false;
|
||||
}
|
||||
|
||||
let mut keep_alive = self.timeouts.keep_alive.is_some() &&
|
||||
http::should_keep_alive(req.version, &req.headers);
|
||||
let version = req.version;
|
||||
let mut res_headers = Headers::new();
|
||||
if !keep_alive {
|
||||
res_headers.set(Connection::close());
|
||||
}
|
||||
let mut config = rotor::Config::new();
|
||||
config.slab_capacity(self.max_sockets);
|
||||
config.mio().notify_capacity(self.max_sockets);
|
||||
let keep_alive = self.keep_alive;
|
||||
let mut loop_ = rotor::Loop::new(&config).unwrap();
|
||||
let mut notifier = None;
|
||||
{
|
||||
let mut res = Response::new(wrt, &mut res_headers);
|
||||
res.version = version;
|
||||
self.handler.handle(req, res);
|
||||
let notifier = &mut notifier;
|
||||
loop_.add_machine_with(move |scope| {
|
||||
*notifier = Some(scope.notifier());
|
||||
rotor_try!(scope.register(&self.listener, EventSet::readable(), PollOpt::level()));
|
||||
rotor::Response::ok(ServerFsm::Listener::<A, H>(self.listener, shutdown_rx))
|
||||
}).unwrap();
|
||||
}
|
||||
let notifier = notifier.expect("loop.add_machine failed");
|
||||
|
||||
// if the request was keep-alive, we need to check that the server agrees
|
||||
// if it wasn't, then the server cannot force it to be true anyways
|
||||
if keep_alive {
|
||||
keep_alive = http::should_keep_alive(version, &res_headers);
|
||||
}
|
||||
let listening = Listening {
|
||||
addr: addr,
|
||||
shutdown: (shutdown, notifier),
|
||||
};
|
||||
let server = ServerLoop {
|
||||
inner: Some((loop_, Context {
|
||||
keep_alive: keep_alive,
|
||||
factory: factory
|
||||
}))
|
||||
};
|
||||
Ok((listening, server))
|
||||
}
|
||||
}
|
||||
|
||||
debug!("keep_alive = {:?} for {}", keep_alive, addr);
|
||||
keep_alive
|
||||
|
||||
impl<A: Accept, H: HandlerFactory<A::Output>> ServerLoop<A, H> {
|
||||
/// Runs the server forever in this loop.
|
||||
///
|
||||
/// This will block the current thread.
|
||||
pub fn run(self) {
|
||||
// drop will take care of it.
|
||||
}
|
||||
}
|
||||
|
||||
impl<A: Accept, H: HandlerFactory<A::Output>> Drop for ServerLoop<A, H> {
|
||||
fn drop(&mut self) {
|
||||
self.inner.take().map(|(loop_, ctx)| {
|
||||
let _ = loop_.run(ctx);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
struct Context<F> {
|
||||
keep_alive: bool,
|
||||
factory: F,
|
||||
}
|
||||
|
||||
impl<F: HandlerFactory<T>, T: Transport> http::MessageHandlerFactory<(), T> for Context<F> {
|
||||
type Output = message::Message<F::Output, T>;
|
||||
|
||||
fn create(&mut self, seed: http::Seed<()>) -> Self::Output {
|
||||
message::Message::new(self.factory.create(seed.control()))
|
||||
}
|
||||
}
|
||||
|
||||
enum ServerFsm<A, H>
|
||||
where A: Accept,
|
||||
A::Output: Transport,
|
||||
H: HandlerFactory<A::Output> {
|
||||
Listener(A, Arc<AtomicBool>),
|
||||
Conn(http::Conn<(), A::Output, message::Message<H::Output, A::Output>>)
|
||||
}
|
||||
|
||||
impl<A, H> rotor::Machine for ServerFsm<A, H>
|
||||
where A: Accept,
|
||||
A::Output: Transport,
|
||||
H: HandlerFactory<A::Output> {
|
||||
type Context = Context<H>;
|
||||
type Seed = A::Output;
|
||||
|
||||
fn create(seed: Self::Seed, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, rotor::Void> {
|
||||
rotor_try!(scope.register(&seed, EventSet::readable(), PollOpt::level()));
|
||||
rotor::Response::ok(
|
||||
ServerFsm::Conn(
|
||||
http::Conn::new((), seed, scope.notifier())
|
||||
.keep_alive(scope.keep_alive)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
fn handle_expect<W: Write>(&self, req: &Request, wrt: &mut W) -> bool {
|
||||
if req.version == Http11 && req.headers.get() == Some(&Expect::Continue) {
|
||||
let status = self.handler.check_continue((&req.method, &req.uri, &req.headers));
|
||||
match write!(wrt, "{} {}\r\n\r\n", Http11, status).and_then(|_| wrt.flush()) {
|
||||
Ok(..) => (),
|
||||
Err(e) => {
|
||||
error!("error writing 100-continue: {:?}", e);
|
||||
return false;
|
||||
fn ready(self, events: EventSet, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> {
|
||||
match self {
|
||||
ServerFsm::Listener(listener, rx) => {
|
||||
match listener.accept() {
|
||||
Ok(Some(conn)) => {
|
||||
rotor::Response::spawn(ServerFsm::Listener(listener, rx), conn)
|
||||
},
|
||||
Ok(None) => rotor::Response::ok(ServerFsm::Listener(listener, rx)),
|
||||
Err(e) => {
|
||||
error!("listener accept error {}", e);
|
||||
// usually fine, just keep listening
|
||||
rotor::Response::ok(ServerFsm::Listener(listener, rx))
|
||||
}
|
||||
}
|
||||
},
|
||||
ServerFsm::Conn(conn) => {
|
||||
match conn.ready(events, scope) {
|
||||
Some((conn, None)) => rotor::Response::ok(ServerFsm::Conn(conn)),
|
||||
Some((conn, Some(dur))) => {
|
||||
rotor::Response::ok(ServerFsm::Conn(conn))
|
||||
.deadline(scope.now() + dur)
|
||||
}
|
||||
None => rotor::Response::done()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if status != StatusCode::Continue {
|
||||
debug!("non-100 status ({}) for Expect 100 request", status);
|
||||
return false;
|
||||
}
|
||||
fn spawned(self, _scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> {
|
||||
match self {
|
||||
ServerFsm::Listener(listener, rx) => {
|
||||
match listener.accept() {
|
||||
Ok(Some(conn)) => {
|
||||
rotor::Response::spawn(ServerFsm::Listener(listener, rx), conn)
|
||||
},
|
||||
Ok(None) => rotor::Response::ok(ServerFsm::Listener(listener, rx)),
|
||||
Err(e) => {
|
||||
error!("listener accept error {}", e);
|
||||
// usually fine, just keep listening
|
||||
rotor::Response::ok(ServerFsm::Listener(listener, rx))
|
||||
}
|
||||
}
|
||||
},
|
||||
sock => rotor::Response::ok(sock)
|
||||
}
|
||||
|
||||
true
|
||||
}
|
||||
|
||||
fn timeout(self, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> {
|
||||
match self {
|
||||
ServerFsm::Listener(..) => unreachable!("Listener cannot timeout"),
|
||||
ServerFsm::Conn(conn) => {
|
||||
match conn.timeout(scope) {
|
||||
Some((conn, None)) => rotor::Response::ok(ServerFsm::Conn(conn)),
|
||||
Some((conn, Some(dur))) => {
|
||||
rotor::Response::ok(ServerFsm::Conn(conn))
|
||||
.deadline(scope.now() + dur)
|
||||
}
|
||||
None => rotor::Response::done()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn wakeup(self, scope: &mut Scope<Self::Context>) -> rotor::Response<Self, Self::Seed> {
|
||||
match self {
|
||||
ServerFsm::Listener(lst, shutdown) => {
|
||||
if shutdown.load(Ordering::Acquire) {
|
||||
let _ = scope.deregister(&lst);
|
||||
scope.shutdown_loop();
|
||||
rotor::Response::done()
|
||||
} else {
|
||||
rotor::Response::ok(ServerFsm::Listener(lst, shutdown))
|
||||
}
|
||||
},
|
||||
ServerFsm::Conn(conn) => match conn.wakeup(scope) {
|
||||
Some((conn, None)) => rotor::Response::ok(ServerFsm::Conn(conn)),
|
||||
Some((conn, Some(dur))) => {
|
||||
rotor::Response::ok(ServerFsm::Conn(conn))
|
||||
.deadline(scope.now() + dur)
|
||||
}
|
||||
None => rotor::Response::done()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A listening server, which can later be closed.
|
||||
/// A handle of the running server.
|
||||
pub struct Listening {
|
||||
_guard: Option<JoinHandle<()>>,
|
||||
/// The socket addresses that the server is bound to.
|
||||
pub socket: SocketAddr,
|
||||
addr: SocketAddr,
|
||||
shutdown: (Arc<AtomicBool>, rotor::Notifier),
|
||||
}
|
||||
|
||||
impl fmt::Debug for Listening {
|
||||
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
|
||||
write!(f, "Listening {{ socket: {:?} }}", self.socket)
|
||||
f.debug_struct("Listening")
|
||||
.field("addr", &self.addr)
|
||||
.field("closed", &self.shutdown.0.load(Ordering::Relaxed))
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for Listening {
|
||||
fn drop(&mut self) {
|
||||
let _ = self._guard.take().map(|g| g.join());
|
||||
impl fmt::Display for Listening {
|
||||
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
|
||||
fmt::Display::fmt(&self.addr, f)
|
||||
}
|
||||
}
|
||||
|
||||
impl Listening {
|
||||
/// Warning: This function doesn't work. The server remains listening after you called
|
||||
/// it. See https://github.com/hyperium/hyper/issues/338 for more details.
|
||||
///
|
||||
/// The address this server is listening on.
|
||||
pub fn addr(&self) -> &SocketAddr {
|
||||
&self.addr
|
||||
}
|
||||
|
||||
/// Stop the server from listening to its socket address.
|
||||
pub fn close(&mut self) -> ::Result<()> {
|
||||
let _ = self._guard.take();
|
||||
debug!("closing server");
|
||||
Ok(())
|
||||
pub fn close(self) {
|
||||
debug!("closing server {}", self);
|
||||
self.shutdown.0.store(true, Ordering::Release);
|
||||
self.shutdown.1.wakeup().unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
/// A handler that can handle incoming requests for a server.
|
||||
pub trait Handler: Sync + Send {
|
||||
/// Receives a `Request`/`Response` pair, and should perform some action on them.
|
||||
///
|
||||
/// This could reading from the request, and writing to the response.
|
||||
fn handle<'a, 'k>(&'a self, Request<'a, 'k>, Response<'a, Fresh>);
|
||||
/// A trait to react to server events that happen for each message.
|
||||
///
|
||||
/// Each event handler returns it's desired `Next` action.
|
||||
pub trait Handler<T: Transport> {
|
||||
/// This event occurs first, triggering when a `Request` has been parsed.
|
||||
fn on_request(&mut self, request: Request) -> Next;
|
||||
/// This event occurs each time the `Request` is ready to be read from.
|
||||
fn on_request_readable(&mut self, request: &mut http::Decoder<T>) -> Next;
|
||||
/// This event occurs after the first time this handled signals `Next::write()`.
|
||||
fn on_response(&mut self, response: &mut Response) -> Next;
|
||||
/// This event occurs each time the `Response` is ready to be written to.
|
||||
fn on_response_writable(&mut self, response: &mut http::Encoder<T>) -> Next;
|
||||
|
||||
/// Called when a Request includes a `Expect: 100-continue` header.
|
||||
/// This event occurs whenever an `Error` occurs outside of the other events.
|
||||
///
|
||||
/// By default, this will always immediately response with a `StatusCode::Continue`,
|
||||
/// but can be overridden with custom behavior.
|
||||
fn check_continue(&self, _: (&Method, &RequestUri, &Headers)) -> StatusCode {
|
||||
StatusCode::Continue
|
||||
/// This could IO errors while waiting for events, or a timeout, etc.
|
||||
fn on_error(&mut self, err: ::Error) -> Next where Self: Sized {
|
||||
debug!("default Handler.on_error({:?})", err);
|
||||
http::Next::remove()
|
||||
}
|
||||
|
||||
/// This is run after a connection is received, on a per-connection basis (not a
|
||||
/// per-request basis, as a connection with keep-alive may handle multiple
|
||||
/// requests)
|
||||
fn on_connection_start(&self) { }
|
||||
|
||||
/// This is run before a connection is closed, on a per-connection basis (not a
|
||||
/// per-request basis, as a connection with keep-alive may handle multiple
|
||||
/// requests)
|
||||
fn on_connection_end(&self) { }
|
||||
/// 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");
|
||||
}
|
||||
}
|
||||
|
||||
impl<F> Handler for F where F: Fn(Request, Response<Fresh>), F: Sync + Send {
|
||||
fn handle<'a, 'k>(&'a self, req: Request<'a, 'k>, res: Response<'a, Fresh>) {
|
||||
self(req, res)
|
||||
|
||||
/// Used to create a `Handler` when a new message is received by the server.
|
||||
pub trait HandlerFactory<T: Transport> {
|
||||
/// The `Handler` to use for the incoming message.
|
||||
type Output: Handler<T>;
|
||||
/// Creates the associated `Handler`.
|
||||
fn create(&mut self, ctrl: http::Control) -> Self::Output;
|
||||
}
|
||||
|
||||
impl<F, H, T> HandlerFactory<T> for F
|
||||
where F: FnMut(http::Control) -> H, H: Handler<T>, T: Transport {
|
||||
type Output = H;
|
||||
fn create(&mut self, ctrl: http::Control) -> H {
|
||||
self(ctrl)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use header::Headers;
|
||||
use method::Method;
|
||||
use mock::MockStream;
|
||||
use status::StatusCode;
|
||||
use uri::RequestUri;
|
||||
|
||||
use super::{Request, Response, Fresh, Handler, Worker};
|
||||
|
||||
#[test]
|
||||
fn test_check_continue_default() {
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
POST /upload HTTP/1.1\r\n\
|
||||
Host: example.domain\r\n\
|
||||
Expect: 100-continue\r\n\
|
||||
Content-Length: 10\r\n\
|
||||
\r\n\
|
||||
1234567890\
|
||||
");
|
||||
|
||||
fn handle(_: Request, res: Response<Fresh>) {
|
||||
res.start().unwrap().end().unwrap();
|
||||
}
|
||||
|
||||
Worker::new(handle, Default::default()).handle_connection(&mut mock);
|
||||
let cont = b"HTTP/1.1 100 Continue\r\n\r\n";
|
||||
assert_eq!(&mock.write[..cont.len()], cont);
|
||||
let res = b"HTTP/1.1 200 OK\r\n";
|
||||
assert_eq!(&mock.write[cont.len()..cont.len() + res.len()], res);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_check_continue_reject() {
|
||||
struct Reject;
|
||||
impl Handler for Reject {
|
||||
fn handle<'a, 'k>(&'a self, _: Request<'a, 'k>, res: Response<'a, Fresh>) {
|
||||
res.start().unwrap().end().unwrap();
|
||||
}
|
||||
|
||||
fn check_continue(&self, _: (&Method, &RequestUri, &Headers)) -> StatusCode {
|
||||
StatusCode::ExpectationFailed
|
||||
}
|
||||
}
|
||||
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
POST /upload HTTP/1.1\r\n\
|
||||
Host: example.domain\r\n\
|
||||
Expect: 100-continue\r\n\
|
||||
Content-Length: 10\r\n\
|
||||
\r\n\
|
||||
1234567890\
|
||||
");
|
||||
|
||||
Worker::new(Reject, Default::default()).handle_connection(&mut mock);
|
||||
assert_eq!(mock.write, &b"HTTP/1.1 417 Expectation Failed\r\n\r\n"[..]);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -2,324 +2,75 @@
|
||||
//!
|
||||
//! These are requests that a `hyper::Server` receives, and include its method,
|
||||
//! target URI, headers, and message body.
|
||||
use std::io::{self, Read};
|
||||
use std::net::SocketAddr;
|
||||
use std::time::Duration;
|
||||
//use std::net::SocketAddr;
|
||||
|
||||
use buffer::BufReader;
|
||||
use net::NetworkStream;
|
||||
use version::{HttpVersion};
|
||||
use version::HttpVersion;
|
||||
use method::Method;
|
||||
use header::{Headers, ContentLength, TransferEncoding};
|
||||
use http::h1::{self, Incoming, HttpReader};
|
||||
use http::h1::HttpReader::{SizedReader, ChunkedReader, EmptyReader};
|
||||
use header::Headers;
|
||||
use http::{RequestHead, MessageHead, RequestLine};
|
||||
use uri::RequestUri;
|
||||
|
||||
pub fn new(incoming: RequestHead) -> Request {
|
||||
let MessageHead { version, subject: RequestLine(method, uri), headers } = incoming;
|
||||
debug!("Request Line: {:?} {:?} {:?}", method, uri, version);
|
||||
debug!("{:#?}", headers);
|
||||
|
||||
Request {
|
||||
//remote_addr: addr,
|
||||
method: method,
|
||||
uri: uri,
|
||||
headers: headers,
|
||||
version: version,
|
||||
}
|
||||
}
|
||||
|
||||
/// A request bundles several parts of an incoming `NetworkStream`, given to a `Handler`.
|
||||
pub struct Request<'a, 'b: 'a> {
|
||||
/// The IP address of the remote connection.
|
||||
pub remote_addr: SocketAddr,
|
||||
/// The `Method`, such as `Get`, `Post`, etc.
|
||||
pub method: Method,
|
||||
/// The headers of the incoming request.
|
||||
pub headers: Headers,
|
||||
/// The target request-uri for this request.
|
||||
pub uri: RequestUri,
|
||||
/// The version of HTTP for this request.
|
||||
pub version: HttpVersion,
|
||||
body: HttpReader<&'a mut BufReader<&'b mut NetworkStream>>
|
||||
#[derive(Debug)]
|
||||
pub struct Request {
|
||||
// The IP address of the remote connection.
|
||||
//remote_addr: SocketAddr,
|
||||
method: Method,
|
||||
headers: Headers,
|
||||
uri: RequestUri,
|
||||
version: HttpVersion,
|
||||
}
|
||||
|
||||
|
||||
impl<'a, 'b: 'a> Request<'a, 'b> {
|
||||
/// Create a new Request, reading the StartLine and Headers so they are
|
||||
/// immediately useful.
|
||||
pub fn new(mut stream: &'a mut BufReader<&'b mut NetworkStream>, addr: SocketAddr)
|
||||
-> ::Result<Request<'a, 'b>> {
|
||||
|
||||
let Incoming { version, subject: (method, uri), headers } = try!(h1::parse_request(stream));
|
||||
debug!("Request Line: {:?} {:?} {:?}", method, uri, version);
|
||||
debug!("{:?}", headers);
|
||||
|
||||
let body = if headers.has::<ContentLength>() {
|
||||
match headers.get::<ContentLength>() {
|
||||
Some(&ContentLength(len)) => SizedReader(stream, len),
|
||||
None => unreachable!()
|
||||
}
|
||||
} else if headers.has::<TransferEncoding>() {
|
||||
todo!("check for Transfer-Encoding: chunked");
|
||||
ChunkedReader(stream, None)
|
||||
} else {
|
||||
EmptyReader(stream)
|
||||
};
|
||||
|
||||
Ok(Request {
|
||||
remote_addr: addr,
|
||||
method: method,
|
||||
uri: uri,
|
||||
headers: headers,
|
||||
version: version,
|
||||
body: body
|
||||
})
|
||||
}
|
||||
|
||||
/// Set the read timeout of the underlying NetworkStream.
|
||||
impl Request {
|
||||
/// The `Method`, such as `Get`, `Post`, etc.
|
||||
#[inline]
|
||||
pub fn set_read_timeout(&self, timeout: Option<Duration>) -> io::Result<()> {
|
||||
self.body.get_ref().get_ref().set_read_timeout(timeout)
|
||||
}
|
||||
pub fn method(&self) -> &Method { &self.method }
|
||||
|
||||
/// Get a reference to the underlying `NetworkStream`.
|
||||
/// The headers of the incoming request.
|
||||
#[inline]
|
||||
pub fn downcast_ref<T: NetworkStream>(&self) -> Option<&T> {
|
||||
self.body.get_ref().get_ref().downcast_ref()
|
||||
}
|
||||
pub fn headers(&self) -> &Headers { &self.headers }
|
||||
|
||||
/// Get a reference to the underlying Ssl stream, if connected
|
||||
/// over HTTPS.
|
||||
///
|
||||
/// # Example
|
||||
///
|
||||
/// ```rust
|
||||
/// # extern crate hyper;
|
||||
/// # #[cfg(feature = "openssl")]
|
||||
/// extern crate openssl;
|
||||
/// # #[cfg(feature = "openssl")]
|
||||
/// use openssl::ssl::SslStream;
|
||||
/// use hyper::net::HttpStream;
|
||||
/// # fn main() {}
|
||||
/// # #[cfg(feature = "openssl")]
|
||||
/// # fn doc_ssl(req: hyper::server::Request) {
|
||||
/// let maybe_ssl = req.ssl::<SslStream<HttpStream>>();
|
||||
/// # }
|
||||
/// ```
|
||||
/// The target request-uri for this request.
|
||||
#[inline]
|
||||
pub fn ssl<T: NetworkStream>(&self) -> Option<&T> {
|
||||
use ::net::HttpsStream;
|
||||
match self.downcast_ref() {
|
||||
Some(&HttpsStream::Https(ref s)) => Some(s),
|
||||
pub fn uri(&self) -> &RequestUri { &self.uri }
|
||||
|
||||
/// The version of HTTP for this request.
|
||||
#[inline]
|
||||
pub fn version(&self) -> &HttpVersion { &self.version }
|
||||
|
||||
/*
|
||||
/// The target path of this Request.
|
||||
#[inline]
|
||||
pub fn path(&self) -> Option<&str> {
|
||||
match *self.uri {
|
||||
RequestUri::AbsolutePath(ref s) => Some(s),
|
||||
RequestUri::AbsoluteUri(ref url) => Some(&url[::url::Position::BeforePath..]),
|
||||
_ => None
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
/// Deconstruct a Request into its constituent parts.
|
||||
/// Deconstruct this Request into its pieces.
|
||||
///
|
||||
/// Modifying these pieces will have no effect on how hyper behaves.
|
||||
#[inline]
|
||||
pub fn deconstruct(self) -> (SocketAddr, Method, Headers,
|
||||
RequestUri, HttpVersion,
|
||||
HttpReader<&'a mut BufReader<&'b mut NetworkStream>>) {
|
||||
(self.remote_addr, self.method, self.headers,
|
||||
self.uri, self.version, self.body)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'b> Read for Request<'a, 'b> {
|
||||
#[inline]
|
||||
fn read(&mut self, buf: &mut [u8]) -> io::Result<usize> {
|
||||
self.body.read(buf)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use buffer::BufReader;
|
||||
use header::{Host, TransferEncoding, Encoding};
|
||||
use net::NetworkStream;
|
||||
use mock::MockStream;
|
||||
use super::Request;
|
||||
|
||||
use std::io::{self, Read};
|
||||
use std::net::SocketAddr;
|
||||
|
||||
fn sock(s: &str) -> SocketAddr {
|
||||
s.parse().unwrap()
|
||||
}
|
||||
|
||||
fn read_to_string(mut req: Request) -> io::Result<String> {
|
||||
let mut s = String::new();
|
||||
try!(req.read_to_string(&mut s));
|
||||
Ok(s)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_get_empty_body() {
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
GET / HTTP/1.1\r\n\
|
||||
Host: example.domain\r\n\
|
||||
\r\n\
|
||||
I'm a bad request.\r\n\
|
||||
");
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
assert_eq!(read_to_string(req).unwrap(), "".to_owned());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_get_with_body() {
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
GET / HTTP/1.1\r\n\
|
||||
Host: example.domain\r\n\
|
||||
Content-Length: 19\r\n\
|
||||
\r\n\
|
||||
I'm a good request.\r\n\
|
||||
");
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
assert_eq!(read_to_string(req).unwrap(), "I'm a good request.".to_owned());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_head_empty_body() {
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
HEAD / HTTP/1.1\r\n\
|
||||
Host: example.domain\r\n\
|
||||
\r\n\
|
||||
I'm a bad request.\r\n\
|
||||
");
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
assert_eq!(read_to_string(req).unwrap(), "".to_owned());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_post_empty_body() {
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
POST / HTTP/1.1\r\n\
|
||||
Host: example.domain\r\n\
|
||||
\r\n\
|
||||
I'm a bad request.\r\n\
|
||||
");
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
assert_eq!(read_to_string(req).unwrap(), "".to_owned());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_parse_chunked_request() {
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
POST / HTTP/1.1\r\n\
|
||||
Host: example.domain\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"
|
||||
);
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
|
||||
// The headers are correct?
|
||||
match req.headers.get::<Host>() {
|
||||
Some(host) => {
|
||||
assert_eq!("example.domain", host.hostname);
|
||||
},
|
||||
None => panic!("Host header expected!"),
|
||||
};
|
||||
match req.headers.get::<TransferEncoding>() {
|
||||
Some(encodings) => {
|
||||
assert_eq!(1, encodings.len());
|
||||
assert_eq!(Encoding::Chunked, encodings[0]);
|
||||
}
|
||||
None => panic!("Transfer-Encoding: chunked expected!"),
|
||||
};
|
||||
// The content is correctly read?
|
||||
assert_eq!(read_to_string(req).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 mut mock = MockStream::with_input(b"\
|
||||
POST / HTTP/1.1\r\n\
|
||||
Host: example.domain\r\n\
|
||||
Transfer-Encoding: chunked\r\n\
|
||||
\r\n\
|
||||
X\r\n\
|
||||
1\r\n\
|
||||
0\r\n\
|
||||
\r\n"
|
||||
);
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
|
||||
assert!(read_to_string(req).is_err());
|
||||
}
|
||||
|
||||
/// Tests that when a chunk size contains an invalid extension, an error is
|
||||
/// returned.
|
||||
#[test]
|
||||
fn test_invalid_chunk_size_extension() {
|
||||
let mut mock = MockStream::with_input(b"\
|
||||
POST / HTTP/1.1\r\n\
|
||||
Host: example.domain\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"
|
||||
);
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
|
||||
assert!(read_to_string(req).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 mut mock = MockStream::with_input(b"\
|
||||
POST / HTTP/1.1\r\n\
|
||||
Host: example.domain\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"
|
||||
);
|
||||
|
||||
// FIXME: Use Type ascription
|
||||
let mock: &mut NetworkStream = &mut mock;
|
||||
let mut stream = BufReader::new(mock);
|
||||
|
||||
let req = Request::new(&mut stream, sock("127.0.0.1:80")).unwrap();
|
||||
|
||||
assert_eq!(read_to_string(req).unwrap(), "1".to_owned());
|
||||
pub fn deconstruct(self) -> (Method, RequestUri, HttpVersion, Headers) {
|
||||
(self.method, self.uri, self.version, self.headers)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -2,431 +2,49 @@
|
||||
//!
|
||||
//! These are responses sent by a `hyper::Server` to clients, after
|
||||
//! receiving a request.
|
||||
use std::any::{Any, TypeId};
|
||||
use std::marker::PhantomData;
|
||||
use std::mem;
|
||||
use std::io::{self, Write};
|
||||
use std::ptr;
|
||||
use std::thread;
|
||||
|
||||
use time::now_utc;
|
||||
|
||||
use header;
|
||||
use http::h1::{LINE_ENDING, HttpWriter};
|
||||
use http::h1::HttpWriter::{ThroughWriter, ChunkedWriter, SizedWriter, EmptyWriter};
|
||||
use status;
|
||||
use net::{Fresh, Streaming};
|
||||
use http;
|
||||
use status::StatusCode;
|
||||
use version;
|
||||
|
||||
|
||||
/// The outgoing half for a Tcp connection, created by a `Server` and given to a `Handler`.
|
||||
///
|
||||
/// The default `StatusCode` for a `Response` is `200 OK`.
|
||||
///
|
||||
/// There is a `Drop` implementation for `Response` that will automatically
|
||||
/// write the head and flush the body, if the handler has not already done so,
|
||||
/// so that the server doesn't accidentally leave dangling requests.
|
||||
#[derive(Debug)]
|
||||
pub struct Response<'a, W: Any = Fresh> {
|
||||
/// The HTTP version of this response.
|
||||
pub version: version::HttpVersion,
|
||||
// Stream the Response is writing to, not accessible through UnwrittenResponse
|
||||
body: HttpWriter<&'a mut (Write + 'a)>,
|
||||
// The status code for the request.
|
||||
status: status::StatusCode,
|
||||
// The outgoing headers on this response.
|
||||
headers: &'a mut header::Headers,
|
||||
|
||||
_writing: PhantomData<W>
|
||||
pub struct Response<'a> {
|
||||
head: &'a mut http::MessageHead<StatusCode>,
|
||||
}
|
||||
|
||||
impl<'a, W: Any> Response<'a, W> {
|
||||
/// The status of this response.
|
||||
#[inline]
|
||||
pub fn status(&self) -> status::StatusCode { self.status }
|
||||
|
||||
impl<'a> Response<'a> {
|
||||
/// The headers of this response.
|
||||
#[inline]
|
||||
pub fn headers(&self) -> &header::Headers { &*self.headers }
|
||||
pub fn headers(&self) -> &header::Headers { &self.head.headers }
|
||||
|
||||
/// Construct a Response from its constituent parts.
|
||||
/// The status of this response.
|
||||
#[inline]
|
||||
pub fn construct(version: version::HttpVersion,
|
||||
body: HttpWriter<&'a mut (Write + 'a)>,
|
||||
status: status::StatusCode,
|
||||
headers: &'a mut header::Headers) -> Response<'a, Fresh> {
|
||||
Response {
|
||||
status: status,
|
||||
version: version,
|
||||
body: body,
|
||||
headers: headers,
|
||||
_writing: PhantomData,
|
||||
}
|
||||
pub fn status(&self) -> &StatusCode {
|
||||
&self.head.subject
|
||||
}
|
||||
|
||||
/// Deconstruct this Response into its constituent parts.
|
||||
/// The HTTP version of this response.
|
||||
#[inline]
|
||||
pub fn deconstruct(self) -> (version::HttpVersion, HttpWriter<&'a mut (Write + 'a)>,
|
||||
status::StatusCode, &'a mut header::Headers) {
|
||||
unsafe {
|
||||
let parts = (
|
||||
self.version,
|
||||
ptr::read(&self.body),
|
||||
self.status,
|
||||
ptr::read(&self.headers)
|
||||
);
|
||||
mem::forget(self);
|
||||
parts
|
||||
}
|
||||
}
|
||||
|
||||
fn write_head(&mut self) -> io::Result<Body> {
|
||||
debug!("writing head: {:?} {:?}", self.version, self.status);
|
||||
try!(write!(&mut self.body, "{} {}\r\n", self.version, self.status));
|
||||
|
||||
if !self.headers.has::<header::Date>() {
|
||||
self.headers.set(header::Date(header::HttpDate(now_utc())));
|
||||
}
|
||||
|
||||
let body_type = match self.status {
|
||||
status::StatusCode::NoContent | status::StatusCode::NotModified => Body::Empty,
|
||||
c if c.class() == status::StatusClass::Informational => Body::Empty,
|
||||
_ => if let Some(cl) = self.headers.get::<header::ContentLength>() {
|
||||
Body::Sized(**cl)
|
||||
} else {
|
||||
Body::Chunked
|
||||
}
|
||||
};
|
||||
|
||||
// can't do in match above, thanks borrowck
|
||||
if body_type == Body::Chunked {
|
||||
let encodings = match self.headers.get_mut::<header::TransferEncoding>() {
|
||||
Some(&mut header::TransferEncoding(ref mut encodings)) => {
|
||||
//TODO: check if chunked is already in encodings. use HashSet?
|
||||
encodings.push(header::Encoding::Chunked);
|
||||
false
|
||||
},
|
||||
None => true
|
||||
};
|
||||
|
||||
if encodings {
|
||||
self.headers.set::<header::TransferEncoding>(
|
||||
header::TransferEncoding(vec![header::Encoding::Chunked]))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
debug!("headers [\n{:?}]", self.headers);
|
||||
try!(write!(&mut self.body, "{}", self.headers));
|
||||
try!(write!(&mut self.body, "{}", LINE_ENDING));
|
||||
|
||||
Ok(body_type)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> Response<'a, Fresh> {
|
||||
/// Creates a new Response that can be used to write to a network stream.
|
||||
#[inline]
|
||||
pub fn new(stream: &'a mut (Write + 'a), headers: &'a mut header::Headers) ->
|
||||
Response<'a, Fresh> {
|
||||
Response {
|
||||
status: status::StatusCode::Ok,
|
||||
version: version::HttpVersion::Http11,
|
||||
headers: headers,
|
||||
body: ThroughWriter(stream),
|
||||
_writing: PhantomData,
|
||||
}
|
||||
}
|
||||
|
||||
/// Writes the body and ends the response.
|
||||
///
|
||||
/// This is a shortcut method for when you have a response with a fixed
|
||||
/// size, and would only need a single `write` call normally.
|
||||
///
|
||||
/// # Example
|
||||
///
|
||||
/// ```
|
||||
/// # use hyper::server::Response;
|
||||
/// fn handler(res: Response) {
|
||||
/// res.send(b"Hello World!").unwrap();
|
||||
/// }
|
||||
/// ```
|
||||
///
|
||||
/// The above is the same, but shorter, than the longer:
|
||||
///
|
||||
/// ```
|
||||
/// # use hyper::server::Response;
|
||||
/// use std::io::Write;
|
||||
/// use hyper::header::ContentLength;
|
||||
/// fn handler(mut res: Response) {
|
||||
/// let body = b"Hello World!";
|
||||
/// res.headers_mut().set(ContentLength(body.len() as u64));
|
||||
/// let mut res = res.start().unwrap();
|
||||
/// res.write_all(body).unwrap();
|
||||
/// }
|
||||
/// ```
|
||||
#[inline]
|
||||
pub fn send(mut self, body: &[u8]) -> io::Result<()> {
|
||||
self.headers.set(header::ContentLength(body.len() as u64));
|
||||
let mut stream = try!(self.start());
|
||||
try!(stream.write_all(body));
|
||||
stream.end()
|
||||
}
|
||||
|
||||
/// Consume this Response<Fresh>, writing the Headers and Status and
|
||||
/// creating a Response<Streaming>
|
||||
pub fn start(mut self) -> io::Result<Response<'a, Streaming>> {
|
||||
let body_type = try!(self.write_head());
|
||||
let (version, body, status, headers) = self.deconstruct();
|
||||
let stream = match body_type {
|
||||
Body::Chunked => ChunkedWriter(body.into_inner()),
|
||||
Body::Sized(len) => SizedWriter(body.into_inner(), len),
|
||||
Body::Empty => EmptyWriter(body.into_inner()),
|
||||
};
|
||||
|
||||
// "copy" to change the phantom type
|
||||
Ok(Response {
|
||||
version: version,
|
||||
body: stream,
|
||||
status: status,
|
||||
headers: headers,
|
||||
_writing: PhantomData,
|
||||
})
|
||||
}
|
||||
/// Get a mutable reference to the status.
|
||||
#[inline]
|
||||
pub fn status_mut(&mut self) -> &mut status::StatusCode { &mut self.status }
|
||||
pub fn version(&self) -> &version::HttpVersion { &self.head.version }
|
||||
|
||||
/// Get a mutable reference to the Headers.
|
||||
#[inline]
|
||||
pub fn headers_mut(&mut self) -> &mut header::Headers { self.headers }
|
||||
}
|
||||
pub fn headers_mut(&mut self) -> &mut header::Headers { &mut self.head.headers }
|
||||
|
||||
|
||||
impl<'a> Response<'a, Streaming> {
|
||||
/// Flushes all writing of a response to the client.
|
||||
/// Get a mutable reference to the status.
|
||||
#[inline]
|
||||
pub fn end(self) -> io::Result<()> {
|
||||
trace!("ending");
|
||||
let (_, body, _, _) = self.deconstruct();
|
||||
try!(body.end());
|
||||
Ok(())
|
||||
pub fn set_status(&mut self, status: StatusCode) {
|
||||
self.head.subject = status;
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> Write for Response<'a, Streaming> {
|
||||
#[inline]
|
||||
fn write(&mut self, msg: &[u8]) -> io::Result<usize> {
|
||||
debug!("write {:?} bytes", msg.len());
|
||||
self.body.write(msg)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn flush(&mut self) -> io::Result<()> {
|
||||
self.body.flush()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(PartialEq)]
|
||||
enum Body {
|
||||
Chunked,
|
||||
Sized(u64),
|
||||
Empty,
|
||||
}
|
||||
|
||||
impl<'a, T: Any> Drop for Response<'a, T> {
|
||||
fn drop(&mut self) {
|
||||
if TypeId::of::<T>() == TypeId::of::<Fresh>() {
|
||||
if thread::panicking() {
|
||||
self.status = status::StatusCode::InternalServerError;
|
||||
}
|
||||
|
||||
let mut body = match self.write_head() {
|
||||
Ok(Body::Chunked) => ChunkedWriter(self.body.get_mut()),
|
||||
Ok(Body::Sized(len)) => SizedWriter(self.body.get_mut(), len),
|
||||
Ok(Body::Empty) => EmptyWriter(self.body.get_mut()),
|
||||
Err(e) => {
|
||||
debug!("error dropping request: {:?}", e);
|
||||
return;
|
||||
}
|
||||
};
|
||||
end(&mut body);
|
||||
} else {
|
||||
end(&mut self.body);
|
||||
};
|
||||
|
||||
|
||||
#[inline]
|
||||
fn end<W: Write>(w: &mut W) {
|
||||
match w.write(&[]) {
|
||||
Ok(_) => match w.flush() {
|
||||
Ok(_) => debug!("drop successful"),
|
||||
Err(e) => debug!("error dropping request: {:?}", e)
|
||||
},
|
||||
Err(e) => debug!("error dropping request: {:?}", e)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use header::Headers;
|
||||
use mock::MockStream;
|
||||
use super::Response;
|
||||
|
||||
macro_rules! lines {
|
||||
($s:ident = $($line:pat),+) => ({
|
||||
let s = String::from_utf8($s.write).unwrap();
|
||||
let mut lines = s.split_terminator("\r\n");
|
||||
|
||||
$(
|
||||
match lines.next() {
|
||||
Some($line) => (),
|
||||
other => panic!("line mismatch: {:?} != {:?}", other, stringify!($line))
|
||||
}
|
||||
)+
|
||||
|
||||
assert_eq!(lines.next(), None);
|
||||
})
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_fresh_start() {
|
||||
let mut headers = Headers::new();
|
||||
let mut stream = MockStream::new();
|
||||
{
|
||||
let res = Response::new(&mut stream, &mut headers);
|
||||
res.start().unwrap().deconstruct();
|
||||
}
|
||||
|
||||
lines! { stream =
|
||||
"HTTP/1.1 200 OK",
|
||||
_date,
|
||||
_transfer_encoding,
|
||||
""
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_streaming_end() {
|
||||
let mut headers = Headers::new();
|
||||
let mut stream = MockStream::new();
|
||||
{
|
||||
let res = Response::new(&mut stream, &mut headers);
|
||||
res.start().unwrap().end().unwrap();
|
||||
}
|
||||
|
||||
lines! { stream =
|
||||
"HTTP/1.1 200 OK",
|
||||
_date,
|
||||
_transfer_encoding,
|
||||
"",
|
||||
"0",
|
||||
"" // empty zero body
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_fresh_drop() {
|
||||
use status::StatusCode;
|
||||
let mut headers = Headers::new();
|
||||
let mut stream = MockStream::new();
|
||||
{
|
||||
let mut res = Response::new(&mut stream, &mut headers);
|
||||
*res.status_mut() = StatusCode::NotFound;
|
||||
}
|
||||
|
||||
lines! { stream =
|
||||
"HTTP/1.1 404 Not Found",
|
||||
_date,
|
||||
_transfer_encoding,
|
||||
"",
|
||||
"0",
|
||||
"" // empty zero body
|
||||
}
|
||||
}
|
||||
|
||||
// x86 windows msvc does not support unwinding
|
||||
// See https://github.com/rust-lang/rust/issues/25869
|
||||
#[cfg(not(all(windows, target_arch="x86", target_env="msvc")))]
|
||||
#[test]
|
||||
fn test_fresh_drop_panicing() {
|
||||
use std::thread;
|
||||
use std::sync::{Arc, Mutex};
|
||||
|
||||
use status::StatusCode;
|
||||
|
||||
let stream = MockStream::new();
|
||||
let stream = Arc::new(Mutex::new(stream));
|
||||
let inner_stream = stream.clone();
|
||||
let join_handle = thread::spawn(move || {
|
||||
let mut headers = Headers::new();
|
||||
let mut stream = inner_stream.lock().unwrap();
|
||||
let mut res = Response::new(&mut *stream, &mut headers);
|
||||
*res.status_mut() = StatusCode::NotFound;
|
||||
|
||||
panic!("inside")
|
||||
});
|
||||
|
||||
assert!(join_handle.join().is_err());
|
||||
|
||||
let stream = match stream.lock() {
|
||||
Err(poisoned) => poisoned.into_inner().clone(),
|
||||
Ok(_) => unreachable!()
|
||||
};
|
||||
|
||||
lines! { stream =
|
||||
"HTTP/1.1 500 Internal Server Error",
|
||||
_date,
|
||||
_transfer_encoding,
|
||||
"",
|
||||
"0",
|
||||
"" // empty zero body
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
#[test]
|
||||
fn test_streaming_drop() {
|
||||
use std::io::Write;
|
||||
use status::StatusCode;
|
||||
let mut headers = Headers::new();
|
||||
let mut stream = MockStream::new();
|
||||
{
|
||||
let mut res = Response::new(&mut stream, &mut headers);
|
||||
*res.status_mut() = StatusCode::NotFound;
|
||||
let mut stream = res.start().unwrap();
|
||||
stream.write_all(b"foo").unwrap();
|
||||
}
|
||||
|
||||
lines! { stream =
|
||||
"HTTP/1.1 404 Not Found",
|
||||
_date,
|
||||
_transfer_encoding,
|
||||
"",
|
||||
"3",
|
||||
"foo",
|
||||
"0",
|
||||
"" // empty zero body
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_no_content() {
|
||||
use status::StatusCode;
|
||||
let mut headers = Headers::new();
|
||||
let mut stream = MockStream::new();
|
||||
{
|
||||
let mut res = Response::new(&mut stream, &mut headers);
|
||||
*res.status_mut() = StatusCode::NoContent;
|
||||
res.start().unwrap();
|
||||
}
|
||||
|
||||
lines! { stream =
|
||||
"HTTP/1.1 204 No Content",
|
||||
_date,
|
||||
""
|
||||
}
|
||||
/// Creates a new Response that can be used to write to a network stream.
|
||||
pub fn new<'a>(head: &'a mut http::MessageHead<StatusCode>) -> Response<'a> {
|
||||
Response {
|
||||
head: head
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user