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:
Sean McArthur
2016-05-03 20:45:43 -07:00
parent 1ec56fe6b6
commit d35992d019
65 changed files with 5599 additions and 5023 deletions

View File

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

View File

@@ -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"[..]);
}
}

View File

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

View File

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